From 6b949109f101b8926ba0d0ed2341ca46af989ee1 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 12 Jan 2021 21:46:03 +0300 Subject: [PATCH 001/266] marked places to edit --- src/Storages/StorageReplicatedMergeTree.cpp | 22 +++++++++++---------- src/Storages/StorageReplicatedMergeTree.h | 7 +++++-- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ebf1e43ca04..26348e9b50d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1352,29 +1352,25 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) } bool do_fetch = false; + if (entry.type == LogEntry::GET_PART) { + /// Before fetching the part, we need to look for it in the detached/ folder by checksum. + /// If we are lucky, we do not copy the part over the network. + // TODO do_fetch = true; } else if (entry.type == LogEntry::MERGE_PARTS) - { /// Sometimes it's better to fetch merged part instead of merge /// For example when we don't have all source parts for merge do_fetch = !tryExecuteMerge(entry); - } else if (entry.type == LogEntry::MUTATE_PART) - { /// Sometimes it's better to fetch mutated part instead of merge do_fetch = !tryExecutePartMutation(entry); - } else if (entry.type == LogEntry::ALTER_METADATA) - { return executeMetadataAlter(entry); - } else - { throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type)), ErrorCodes::LOGICAL_ERROR); - } if (do_fetch) return executeFetch(entry); @@ -4460,13 +4456,19 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); - ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); /// TODO Allow to use quorum here. + /// TODO Allow to use quorum here. + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); + for (size_t i = 0; i < loaded_parts.size(); ++i) { - String old_name = loaded_parts[i]->name; + const String old_name = loaded_parts[i]->name; + output.writeExistingPart(loaded_parts[i]); + renamed_parts.old_and_new_names[i].first.clear(); + LOG_DEBUG(log, "Attached part {} as {}", old_name, loaded_parts[i]->name); + results.push_back(PartitionCommandResultInfo{ .partition_id = loaded_parts[i]->info.partition_id, .part_name = loaded_parts[i]->name, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cf36cf82fc9..9c6484e275f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -39,13 +39,14 @@ namespace DB * - the structure of the table (/metadata, /columns) * - action log with data (/log/log-...,/replicas/replica_name/queue/queue-...); * - a replica list (/replicas), and replica activity tag (/replicas/replica_name/is_active), replica addresses (/replicas/replica_name/host); - * - select the leader replica (/leader_election) - these are the replicas that assigning merges, mutations and partition manipulations + * - the leader replica election (/leader_election) - these are the replicas that assign merges, mutations + * and partition manipulations. * (after ClickHouse version 20.5 we allow multiple leaders to act concurrently); * - a set of parts of data on each replica (/replicas/replica_name/parts); * - list of the last N blocks of data with checksum, for deduplication (/blocks); * - the list of incremental block numbers (/block_numbers) that we are about to insert, * to ensure the linear order of data insertion and data merge only on the intervals in this sequence; - * - coordinates writes with quorum (/quorum). + * - coordinate writes with quorum (/quorum). * - Storage of mutation entries (ALTER DELETE, ALTER UPDATE etc.) to execute (/mutations). * See comments in StorageReplicatedMergeTree::mutate() for details. */ @@ -65,6 +66,8 @@ namespace DB * - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check (at start - checkParts, while running - searchForMissingPart), * actions are put on GET from other replicas; * + * TODO Update the GET part after rewriting the code (search locally). + * * The replica to which INSERT was made in the queue will also have an entry of the GET of this data. * Such an entry is considered to be executed as soon as the queue handler sees it. * From feff4c6a22acb0475995d52630562efe069f19cc Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 15 Feb 2021 01:59:13 +0300 Subject: [PATCH 002/266] Started adding the new "ATTACH_PART" command into the replicated log The original ticket idea was to search for the possibly available data into the /detached folders for the GET_PART command, but @tavplubix pointed out this would be quite expensive for an every fetch. So a new command is going to be introduced, ATTACH_PART, which will cover ALTER TABLE ATTACH PART and only for which the search will start. --- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 10 +++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 4 + src/Storages/StorageReplicatedMergeTree.cpp | 88 +++++++++++++++---- 3 files changed, 83 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 9a9f25fd470..9a2fb79002c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -52,6 +52,10 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const out << "get\n" << new_part_name; break; + case ATTACH_PART: + out << "attach\n" << new_part_name; + break; + case MERGE_PARTS: out << "merge\n"; for (const String & s : source_parts) @@ -175,11 +179,17 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> type_str >> "\n"; bool trailing_newline_found = false; + if (type_str == "get") { type = GET_PART; in >> new_part_name; } + else if (type_str == "attach") + { + type = ATTACH_PART; + in >> new_part_name; + } else if (type_str == "merge") { type = MERGE_PARTS; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index cdf5a40d5a9..893a18ae7b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -31,6 +31,7 @@ struct ReplicatedMergeTreeLogEntryData { EMPTY, /// Not used. GET_PART, /// Get the part from another replica. + ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). MERGE_PARTS, /// Merge the parts. DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. @@ -45,6 +46,7 @@ struct ReplicatedMergeTreeLogEntryData switch (type) { case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; + case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; @@ -71,6 +73,8 @@ struct ReplicatedMergeTreeLogEntryData Type type = EMPTY; String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. + String part_checksum; /// Part checksum for ATTACH_PART, empty otherwise. + /// The name of resulting part for GET_PART and MERGE_PARTS /// Part range for DROP_RANGE and CLEAR_COLUMN String new_part_name; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 26348e9b50d..785255c18bd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1330,8 +1330,10 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (!existing_part) existing_part = getActiveContainingPart(entry.new_part_name); - /// Even if the part is locally, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there. - if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name)) + const String part_path = replica_path + "/parts/" + existing_part->name; + + /// Even if the part is local, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there. + if (existing_part && getZooKeeper()->exists(part_path)) { if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)) { @@ -1339,6 +1341,51 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) } return true; } + + auto try_find_part_in_detached = [this, &]() + { + const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); + + MergeTreePartInfo part_iter; + const Poco::DirectoryIterator dir_end; + + // TODO REPLACE + + const String checksum = getZooKeeper()->get(part_path) + + for (const std::string & path : getDataPaths()) + { + for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) + { + if (!MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_iter, format_version) || + part_iter.partition_id != target_part.partition_id) + continue; + + DataPartPtr iter_part_ptr = { + *this, // storage + dir_it.name(), //name + getVolume(), //volume ?? + std::nullopt, // ?? relative path + entry.new_part_type + }; + + // TODO Why hex, not checksums_str? + if (our_part_ptr->checksums.getTotalChecksumHex() != + iter_part_ptr->checksums.getTotalChecksumHex()) + /// the part with same partition id has different checksum, so it is corrupt. + return false; + + ///Attach part + return true; + } + } + }; + + /// We also check for the part in the detached/ folder by checksum + if (entry.type == LogEntry::GET_PART) + { + try_find_part_in_detached(); + } } if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name) @@ -1353,24 +1400,27 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) bool do_fetch = false; - if (entry.type == LogEntry::GET_PART) + switch (entry.type) { - /// Before fetching the part, we need to look for it in the detached/ folder by checksum. - /// If we are lucky, we do not copy the part over the network. - // TODO - do_fetch = true; + case LogEntry::GET_PART: + /// We surely don't have this part locally as we've checked it before, so download it. + do_fetch = true; + break; + case LogEntry::MERGE_PARTS: + /// Sometimes it's better to fetch the merged part instead of merging, + /// e.g when we don't have all the source parts. + do_fetch = !tryExecuteMerge(entry); + break; + case LogEntry::MUTATE_PART: + /// Sometimes it's better to fetch mutated part instead of merging. + do_fetch = !tryExecutePartMutation(entry); + break; + case LogEntry::ALTER_METADATA: + return executeMetadataAlter(entry); + default: + throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type)), + ErrorCodes::LOGICAL_ERROR); } - else if (entry.type == LogEntry::MERGE_PARTS) - /// Sometimes it's better to fetch merged part instead of merge - /// For example when we don't have all source parts for merge - do_fetch = !tryExecuteMerge(entry); - else if (entry.type == LogEntry::MUTATE_PART) - /// Sometimes it's better to fetch mutated part instead of merge - do_fetch = !tryExecutePartMutation(entry); - else if (entry.type == LogEntry::ALTER_METADATA) - return executeMetadataAlter(entry); - else - throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type)), ErrorCodes::LOGICAL_ERROR); if (do_fetch) return executeFetch(entry); @@ -3104,7 +3154,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam for (const String & replica : replicas) { - /// We don't interested in ourself. + /// We aren't interested in ourself. if (replica == replica_name) continue; From 73f4740be5e617ba49be5d21b32ab9a8ba404a1f Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 15 Feb 2021 18:06:48 +0300 Subject: [PATCH 003/266] Trying to re-write the solution by adding new command type ATTACH_PART into the replicated log. The LogEntry now also has the pre-calculated part checksum for this entry type, which is later used while searching in the detached/ folder --- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 17 ++- .../ReplicatedMergeTreeBlockOutputStream.h | 6 +- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 24 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 107 ++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 3 + 7 files changed, 99 insertions(+), 63 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a0d23b8ab22..38e069cdfcc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2694,12 +2694,12 @@ void MergeTreeData::checkAlterPartitionIsPossible(const PartitionCommands & comm if (command.part) { auto part_name = command.partition->as().value.safeGet(); - /// We able to parse it + /// We are able to parse it MergeTreePartInfo::fromPartName(part_name, format_version); } else { - /// We able to parse it + /// We are able to parse it getPartitionIDFromQuery(command.partition, global_context); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 7046a510f75..6b4ad050c3c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -41,7 +41,8 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( size_t max_parts_per_block_, bool quorum_parallel_, bool deduplicate_, - bool optimize_on_insert_) + bool optimize_on_insert_, + bool is_attach_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , quorum(quorum_) @@ -51,6 +52,7 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( , deduplicate(deduplicate_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , optimize_on_insert(optimize_on_insert_) + , is_attach(is_attach_) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) @@ -258,10 +260,17 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->name = part->getNewName(part->info); - /// Will add log entry about new part. - StorageReplicatedMergeTree::LogEntry log_entry; - log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; + + /// Will add log entry about new part. + if (is_attach) + { + log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; + log_entry.part_checksum = ...; //TODO initialize checksum + } + else + log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; + log_entry.create_time = time(nullptr); log_entry.source_replica = storage.replica_name; log_entry.new_part_name = part->name; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 3ac2c4bcfcb..860b0c4ed12 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -30,7 +30,10 @@ public: size_t max_parts_per_block_, bool quorum_parallel_, bool deduplicate_, - bool optimize_on_insert); + bool optimize_on_insert, + // special flag to determine the ALTER TABLE ATTACH PART without the query context, + // needed to set the special LogEntryType::ATTACH_PART + bool is_attach_ = false); Block getHeader() const override; void writePrefix() override; @@ -66,6 +69,7 @@ private: size_t quorum_timeout_ms; size_t max_parts_per_block; + bool is_attach = false; bool quorum_parallel = false; bool deduplicate = true; bool last_block_is_duplicate = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 893a18ae7b3..5e5eb1b0892 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -32,6 +32,7 @@ struct ReplicatedMergeTreeLogEntryData EMPTY, /// Not used. GET_PART, /// Get the part from another replica. ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). + /// You may think of it as a GET_PART with some opmitisations as they're nearly identical. MERGE_PARTS, /// Merge the parts. DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 26a916d2356..ad41bbe1a08 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -145,7 +145,7 @@ void ReplicatedMergeTreeQueue::insertUnlocked( else queue.push_front(entry); - if (entry->type == LogEntry::GET_PART) + if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART) { inserts_by_time.insert(entry); @@ -184,7 +184,7 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( std::unique_lock & state_lock) { /// Update insert times. - if (entry->type == LogEntry::GET_PART) + if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART) { inserts_by_time.erase(entry); @@ -563,7 +563,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper replica_path + "/queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); const auto & entry = *copied_entries.back(); - if (entry.type == LogEntry::GET_PART) + if (entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) { std::lock_guard state_lock(state_mutex); if (entry.create_time && (!min_unprocessed_insert_time || entry.create_time < min_unprocessed_insert_time)) @@ -871,7 +871,12 @@ ReplicatedMergeTreeQueue::StringSet ReplicatedMergeTreeQueue::moveSiblingPartsFo if (it0 == merge_entry) break; - if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART || (*it0)->type == LogEntry::MUTATE_PART) + const auto t = (*it0)->type; + + if ((t == LogEntry::MERGE_PARTS || + t == LogEntry::GET_PART || + t == LogEntry::ATTACH_PART || + t == LogEntry::MUTATE_PART) && parts_for_merge.count((*it0)->new_part_name)) { queue.splice(queue.end(), queue, it0, it); @@ -921,7 +926,10 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( { auto type = (*it)->type; - if (((type == LogEntry::GET_PART || type == LogEntry::MERGE_PARTS || type == LogEntry::MUTATE_PART) + if (((type == LogEntry::GET_PART || + type == LogEntry::ATTACH_PART || + type == LogEntry::MERGE_PARTS || + type == LogEntry::MUTATE_PART) && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version))) || checkReplaceRangeCanBeRemoved(part_info, *it, current)) { @@ -1066,6 +1074,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( /// some other entry which is currently executing, then we can postpone this entry. if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART + || entry.type == LogEntry::ATTACH_PART || entry.type == LogEntry::MUTATE_PART) { for (const String & new_part_name : entry.getBlockingPartNames()) @@ -1076,7 +1085,8 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( } /// Check that fetches pool is not overloaded - if (entry.type == LogEntry::GET_PART && !storage.canExecuteFetch(entry, out_postpone_reason)) + if ((entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) + && !storage.canExecuteFetch(entry, out_postpone_reason)) { /// Don't print log message about this, because we can have a lot of fetches, /// for example during replica recovery. @@ -1643,7 +1653,7 @@ ReplicatedMergeTreeQueue::Status ReplicatedMergeTreeQueue::getStatus() const if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time)) res.queue_oldest_time = entry->create_time; - if (entry->type == LogEntry::GET_PART) + if (entry->type == LogEntry::GET_PART || entry->type == LogEntry::ATTACH_PART) { ++res.inserts_in_queue; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a92eea2c3f3..aead416801d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -54,6 +54,7 @@ #include #include +#include "Storages/MergeTree/MergeTreeReaderCompact.h" #include #include @@ -1303,6 +1304,45 @@ String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataP getSettings()->use_minimalistic_checksums_in_zookeeper); } +MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(const LogEntry& entry) const +{ + const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); + const String& part_checksum = entry.part_checksum; + + MergeTreePartInfo part_iter; + const Poco::DirectoryIterator dir_end; + + for (const String& path : getDataPaths()) + { + for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) + { + if (!MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_iter, format_version) || // this line is correct + part_iter.partition_id != target_part.partition_id || + entry.new_part_name != part_iter.getPartName()) // TODO check if the last statement is valid, + // Maybe we can't compare by names + continue; + + const String& part_name = part_iter.getPartName(); + const String part_dir = "detached/"; //TODO double-check + const String part_to_path = part_dir + part_name; + + auto single_disk_volume = std::make_shared("volume_" + part_name, + getDiskForPart(part_name, part_dir)); + + //createPart uses part name as arg 1, "detached/" as arg 2 so maybe we need "detached/" too + MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); + + if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) + /// the part with same partition id has different checksum, so it is corrupt. + return {}; + + return iter_part_ptr; + } + } + + return {}; +} + bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { @@ -1318,7 +1358,16 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } + /// Try to look in the detached/ folder first, if found, attach the part + if (entry.type == LogEntry::ATTACH_PART) + if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) + // no need to call checkAlterPartitionIsPossible as we already parsed the part name + /// TODO Allow to use quorum here. + ReplicatedMergeTreeBlockOutputStream (*this, getInMemoryMetadataPtr() , 0, 0, 0, false, false, false) + .writeExistingPart(part); + if (entry.type == LogEntry::GET_PART || + entry.type == LogEntry::ATTACH_PART || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) { @@ -1336,61 +1385,19 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)) { - LOG_DEBUG(log, "Skipping action for part {} because part {} already exists.", entry.new_part_name, existing_part->name); + LOG_DEBUG(log, "Skipping action for part {} because part {} already exists.", + entry.new_part_name, existing_part->name); } return true; } - - auto try_find_part_in_detached = [this, &]() - { - const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); - - MergeTreePartInfo part_iter; - const Poco::DirectoryIterator dir_end; - - // TODO REPLACE - - const String checksum = getZooKeeper()->get(part_path) - - for (const std::string & path : getDataPaths()) - { - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) - { - if (!MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_iter, format_version) || - part_iter.partition_id != target_part.partition_id) - continue; - - DataPartPtr iter_part_ptr = { - *this, // storage - dir_it.name(), //name - getVolume(), //volume ?? - std::nullopt, // ?? relative path - entry.new_part_type - }; - - // TODO Why hex, not checksums_str? - if (our_part_ptr->checksums.getTotalChecksumHex() != - iter_part_ptr->checksums.getTotalChecksumHex()) - /// the part with same partition id has different checksum, so it is corrupt. - return false; - - ///Attach part - return true; - } - } - }; - - /// We also check for the part in the detached/ folder by checksum - if (entry.type == LogEntry::GET_PART) - { - try_find_part_in_detached(); - } } - if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name) + if ((entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) && + entry.source_replica == replica_name) LOG_WARNING(log, "Part {} from own log doesn't exist.", entry.new_part_name); - /// Perhaps we don't need this part, because during write with quorum, the quorum has failed (see below about `/quorum/failed_parts`). + /// Perhaps we don't need this part, because during write with quorum, the quorum has failed + /// (see below about `/quorum/failed_parts`). if (entry.quorum && getZooKeeper()->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name)) { LOG_DEBUG(log, "Skipping action for part {} because quorum for that part was failed.", entry.new_part_name); @@ -1401,8 +1408,10 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) switch (entry.type) { - case LogEntry::GET_PART: + case LogEntry::ATTACH_PART: /// We surely don't have this part locally as we've checked it before, so download it. + [[fallthrough]]; + case LogEntry::GET_PART: do_fetch = true; break; case LogEntry::MERGE_PARTS: diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 02c956ff8ce..10d0734910a 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -404,6 +404,9 @@ private: */ bool executeLogEntry(LogEntry & entry); + /// Lookup the part for the entry in the detached/ folder. + /// returns nullptr if the part is corrupt or missing. + MutableDataPartPtr attachPartHelperFoundValidPart(const LogEntry& entry) const; void executeDropRange(const LogEntry & entry); From 0a02fe913a74fb6b9dc3e2e80636230cbc3308b4 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 15 Feb 2021 20:31:58 +0300 Subject: [PATCH 004/266] Add some code for the checksum pre-calculation in ...BlockOutputStream Added the comment explaining the double-get for the zookeeper header for the part. --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 17 ++++++++++++++--- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 4 ++++ src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 6b4ad050c3c..723363c8589 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -262,11 +262,22 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( StorageReplicatedMergeTree::LogEntry log_entry; - /// Will add log entry about new part. if (is_attach) { - log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; - log_entry.part_checksum = ...; //TODO initialize checksum + const String part_path = storage.replica_path + "/parts/" + part->name; + const String part_znode = zookeeper->get(part_path); + + /// We don't support old parts (that were existent before the ReplicatedMergeTreePartHeader + /// introduction), so for them the implementation falls back to fetching from other replica. + /// See also ReplicatedMergeTreePartCheckThread.cpp:253 + if (part_znode.empty()) + log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; + else + { + log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; + log_entry.part_checksum = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); + // TODO + } } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 5e5eb1b0892..b737bffeb28 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -32,7 +32,7 @@ struct ReplicatedMergeTreeLogEntryData EMPTY, /// Not used. GET_PART, /// Get the part from another replica. ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). - /// You may think of it as a GET_PART with some opmitisations as they're nearly identical. + /// You may think of it as a GET_PART with some optimisations as they're nearly identical. MERGE_PARTS, /// Merge the parts. DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index b2a144ca748..100379c55c4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -250,6 +250,10 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); + /// The double-get scheme is needed to retain compatibility with very old parts that were created + /// before the ReplicatedMergeTreePartHeader was introduced. + /// See also ReplicatedMergeTreeBlockOutputStream.cpp:270 + String part_path = storage.replica_path + "/parts/" + part_name; String part_znode; /// If the part is in ZooKeeper, check its data with its checksums, and them with ZooKeeper. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index aead416801d..8eae20a566c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4535,7 +4535,8 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. - ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false); + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false, false, false, + /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) { From 8182482cbd6de6ef799cba4f3d09bf80da0e9ef7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 15 Feb 2021 21:06:20 +0300 Subject: [PATCH 005/266] Add test stub --- src/Storages/StorageReplicatedMergeTree.cpp | 13 +++--- .../test_attach_without_fetching/__init__.py | 0 .../test_attach_without_fetching/test.py | 46 +++++++++++++++++++ 3 files changed, 53 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_attach_without_fetching/__init__.py create mode 100644 tests/integration/test_attach_without_fetching/test.py diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8eae20a566c..c007c2bc4cd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1312,28 +1312,29 @@ MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(co MergeTreePartInfo part_iter; const Poco::DirectoryIterator dir_end; + const String detached_dir = "detached"; + for (const String& path : getDataPaths()) { - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) + for (Poco::DirectoryIterator it{path + detached_dir}; it != dir_end; ++it) { - if (!MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_iter, format_version) || // this line is correct + if (!MergeTreePartInfo::tryParsePartName(it.name(), &part_iter, format_version) || // this line is correct part_iter.partition_id != target_part.partition_id || entry.new_part_name != part_iter.getPartName()) // TODO check if the last statement is valid, // Maybe we can't compare by names continue; const String& part_name = part_iter.getPartName(); - const String part_dir = "detached/"; //TODO double-check - const String part_to_path = part_dir + part_name; + const String part_to_path = detached_dir + part_name; auto single_disk_volume = std::make_shared("volume_" + part_name, - getDiskForPart(part_name, part_dir)); + getDiskForPart(part_name, detached_dir)); //createPart uses part name as arg 1, "detached/" as arg 2 so maybe we need "detached/" too MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) - /// the part with same partition id has different checksum, so it is corrupt. + /// the part with same name and partition id has different checksum, so it is corrupt. return {}; return iter_part_ptr; diff --git a/tests/integration/test_attach_without_fetching/__init__.py b/tests/integration/test_attach_without_fetching/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py new file mode 100644 index 00000000000..637bb0aa1ac --- /dev/null +++ b/tests/integration/test_attach_without_fetching/test.py @@ -0,0 +1,46 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1') + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +# Check that ALTER TABLE ATTACH PARTITION does not fetch data from other replicas if it's present in the +# detached/ folder +def test_attach_without_fetching(start_cluster): + node1.query( + "CREATE TABLE test (date Date, key Int32, value String) Engine=MergeTree ORDER BY key PARTITION by date") + + node1.query("INSERT INTO test SELECT toDate('2019-10-01'), number, toString(number) FROM numbers(100)") + + assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n" + + node1.query("ALTER TABLE test DETACH PARTITION '2019-10-01'") + + assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "0\n" + assert node1.query("SELECT COUNT() FROM test") == "0\n" + + # to be sure output not empty + node1.exec_in_container( + ['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '], + privileged=True, user='root') + + node1.exec_in_container( + ['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" -delete'], + privileged=True, user='root') + + node1.query("ALTER TABLE test ATTACH PARTITION '2019-10-01'") + + assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n" + assert node1.query("SELECT COUNT() FROM test") == "100\n" From a1fafc47c66e5e55727c8b9f62ae2cf71401bffc Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 16 Feb 2021 15:40:00 +0300 Subject: [PATCH 006/266] Correct checksums pre-calculation for ATTACH_PART in ...OutputStream --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 18 +++++------------- .../ReplicatedMergeTreePartCheckThread.cpp | 5 ++--- 2 files changed, 7 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 723363c8589..dd65a67698d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -264,20 +264,12 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( if (is_attach) { - const String part_path = storage.replica_path + "/parts/" + part->name; - const String part_znode = zookeeper->get(part_path); + log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; - /// We don't support old parts (that were existent before the ReplicatedMergeTreePartHeader - /// introduction), so for them the implementation falls back to fetching from other replica. - /// See also ReplicatedMergeTreePartCheckThread.cpp:253 - if (part_znode.empty()) - log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; - else - { - log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; - log_entry.part_checksum = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); - // TODO - } + /// We don't need to involve ZooKeeper to obtain the checksums as by the time we get + /// the MutableDataPartPtr here, we already have the data thus being able to + /// calculate the checksums. + log_entry.part_checksum = part->checksums.getTotalChecksumHex(); } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 100379c55c4..0d845f6b055 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -1,4 +1,4 @@ -#include +2include #include #include #include @@ -250,9 +250,8 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); - /// The double-get scheme is needed to retain compatibility with very old parts that were created + /// The double get scheme is needed to retain compatibility with very old parts that were created /// before the ReplicatedMergeTreePartHeader was introduced. - /// See also ReplicatedMergeTreeBlockOutputStream.cpp:270 String part_path = storage.replica_path + "/parts/" + part_name; String part_znode; From fefc7234df64b83c8f0b3145c093dde16a448d12 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 16 Feb 2021 16:00:26 +0300 Subject: [PATCH 007/266] Replaced the part lookup algo to "by hash only", comments on test stub --- src/Storages/StorageReplicatedMergeTree.cpp | 16 +++++++--------- .../test_attach_without_fetching/test.py | 3 +++ 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c007c2bc4cd..4ebbecc42d6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1312,30 +1312,29 @@ MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(co MergeTreePartInfo part_iter; const Poco::DirectoryIterator dir_end; - const String detached_dir = "detached"; + const String detached_dir = "detached/"; for (const String& path : getDataPaths()) { for (Poco::DirectoryIterator it{path + detached_dir}; it != dir_end; ++it) { - if (!MergeTreePartInfo::tryParsePartName(it.name(), &part_iter, format_version) || // this line is correct - part_iter.partition_id != target_part.partition_id || - entry.new_part_name != part_iter.getPartName()) // TODO check if the last statement is valid, - // Maybe we can't compare by names + if (!MergeTreePartInfo::tryParsePartName(it.name(), &part_iter, format_version) || + part_iter.partition_id != target_part.partition_id) continue; + // TODO Check if we can compare by entry.new_part_name ?== part_iter.getPartName(), + // mostly sure we can't, but if we can, all this thing would work faster. + const String& part_name = part_iter.getPartName(); const String part_to_path = detached_dir + part_name; auto single_disk_volume = std::make_shared("volume_" + part_name, getDiskForPart(part_name, detached_dir)); - //createPart uses part name as arg 1, "detached/" as arg 2 so maybe we need "detached/" too MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) - /// the part with same name and partition id has different checksum, so it is corrupt. - return {}; + continue; // TODO if we can, here would be return {}; return iter_part_ptr; } @@ -1363,7 +1362,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (entry.type == LogEntry::ATTACH_PART) if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) // no need to call checkAlterPartitionIsPossible as we already parsed the part name - /// TODO Allow to use quorum here. ReplicatedMergeTreeBlockOutputStream (*this, getInMemoryMetadataPtr() , 0, 0, 0, false, false, false) .writeExistingPart(part); diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 637bb0aa1ac..d712cf5d2e3 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -31,6 +31,9 @@ def test_attach_without_fetching(start_cluster): assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "0\n" assert node1.query("SELECT COUNT() FROM test") == "0\n" + # Break the network in the partition manager + # The data is not removed from detached/ so it's ok + # to be sure output not empty node1.exec_in_container( ['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '], From ca837757119f20fa73d0ba591d42ca5d008a5722 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 16 Feb 2021 16:19:21 +0300 Subject: [PATCH 008/266] Multiple small hotfixes Small fixes Some fix for old bug Another old code fix --- src/Parsers/New/ClickHouseLexer.h | 82 +++++++++---------- src/Parsers/New/ClickHouseParser.h | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 2 + 6 files changed, 48 insertions(+), 46 deletions(-) diff --git a/src/Parsers/New/ClickHouseLexer.h b/src/Parsers/New/ClickHouseLexer.h index e925c5d271f..62de0792f3c 100644 --- a/src/Parsers/New/ClickHouseLexer.h +++ b/src/Parsers/New/ClickHouseLexer.h @@ -13,51 +13,51 @@ namespace DB { class ClickHouseLexer : public antlr4::Lexer { public: enum { - ADD = 1, AFTER = 2, ALIAS = 3, ALL = 4, ALTER = 5, AND = 6, ANTI = 7, - ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, ASYNC = 13, - ATTACH = 14, BETWEEN = 15, BOTH = 16, BY = 17, CASE = 18, CAST = 19, - CHECK = 20, CLEAR = 21, CLUSTER = 22, CODEC = 23, COLLATE = 24, COLUMN = 25, - COMMENT = 26, CONSTRAINT = 27, CREATE = 28, CROSS = 29, CUBE = 30, DATABASE = 31, - DATABASES = 32, DATE = 33, DAY = 34, DEDUPLICATE = 35, DEFAULT = 36, - DELAY = 37, DELETE = 38, DESC = 39, DESCENDING = 40, DESCRIBE = 41, - DETACH = 42, DICTIONARIES = 43, DICTIONARY = 44, DISK = 45, DISTINCT = 46, - DISTRIBUTED = 47, DROP = 48, ELSE = 49, END = 50, ENGINE = 51, EVENTS = 52, - EXISTS = 53, EXPLAIN = 54, EXPRESSION = 55, EXTRACT = 56, FETCHES = 57, - FINAL = 58, FIRST = 59, FLUSH = 60, FOR = 61, FORMAT = 62, FREEZE = 63, - FROM = 64, FULL = 65, FUNCTION = 66, GLOBAL = 67, GRANULARITY = 68, - GROUP = 69, HAVING = 70, HIERARCHICAL = 71, HOUR = 72, ID = 73, IF = 74, - ILIKE = 75, IN = 76, INDEX = 77, INF = 78, INJECTIVE = 79, INNER = 80, - INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86, - KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92, - LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98, - MATERIALIZED = 99, MAX = 100, MERGES = 101, MIN = 102, MINUTE = 103, - MODIFY = 104, MONTH = 105, MOVE = 106, MUTATION = 107, NAN_SQL = 108, - NO = 109, NOT = 110, NULL_SQL = 111, NULLS = 112, OFFSET = 113, ON = 114, - OPTIMIZE = 115, OR = 116, ORDER = 117, OUTER = 118, OUTFILE = 119, PARTITION = 120, - POPULATE = 121, PREWHERE = 122, PRIMARY = 123, QUARTER = 124, RANGE = 125, - RELOAD = 126, REMOVE = 127, RENAME = 128, REPLACE = 129, REPLICA = 130, - REPLICATED = 131, RIGHT = 132, ROLLUP = 133, SAMPLE = 134, SECOND = 135, - SELECT = 136, SEMI = 137, SENDS = 138, SET = 139, SETTINGS = 140, SHOW = 141, - SOURCE = 142, START = 143, STOP = 144, SUBSTRING = 145, SYNC = 146, - SYNTAX = 147, SYSTEM = 148, TABLE = 149, TABLES = 150, TEMPORARY = 151, - TEST = 152, THEN = 153, TIES = 154, TIMEOUT = 155, TIMESTAMP = 156, - TO = 157, TOP = 158, TOTALS = 159, TRAILING = 160, TRIM = 161, TRUNCATE = 162, - TTL = 163, TYPE = 164, UNION = 165, UPDATE = 166, USE = 167, USING = 168, - UUID = 169, VALUES = 170, VIEW = 171, VOLUME = 172, WATCH = 173, WEEK = 174, - WHEN = 175, WHERE = 176, WITH = 177, YEAR = 178, JSON_FALSE = 179, JSON_TRUE = 180, - IDENTIFIER = 181, FLOATING_LITERAL = 182, OCTAL_LITERAL = 183, DECIMAL_LITERAL = 184, - HEXADECIMAL_LITERAL = 185, STRING_LITERAL = 186, ARROW = 187, ASTERISK = 188, - BACKQUOTE = 189, BACKSLASH = 190, COLON = 191, COMMA = 192, CONCAT = 193, - DASH = 194, DOT = 195, EQ_DOUBLE = 196, EQ_SINGLE = 197, GE = 198, GT = 199, - LBRACE = 200, LBRACKET = 201, LE = 202, LPAREN = 203, LT = 204, NOT_EQ = 205, - PERCENT = 206, PLUS = 207, QUERY = 208, QUOTE_DOUBLE = 209, QUOTE_SINGLE = 210, - RBRACE = 211, RBRACKET = 212, RPAREN = 213, SEMICOLON = 214, SLASH = 215, - UNDERSCORE = 216, MULTI_LINE_COMMENT = 217, SINGLE_LINE_COMMENT = 218, + ADD = 1, AFTER = 2, ALIAS = 3, ALL = 4, ALTER = 5, AND = 6, ANTI = 7, + ANY = 8, ARRAY = 9, AS = 10, ASCENDING = 11, ASOF = 12, ASYNC = 13, + ATTACH = 14, BETWEEN = 15, BOTH = 16, BY = 17, CASE = 18, CAST = 19, + CHECK = 20, CLEAR = 21, CLUSTER = 22, CODEC = 23, COLLATE = 24, COLUMN = 25, + COMMENT = 26, CONSTRAINT = 27, CREATE = 28, CROSS = 29, CUBE = 30, DATABASE = 31, + DATABASES = 32, DATE = 33, DAY = 34, DEDUPLICATE = 35, DEFAULT = 36, + DELAY = 37, DELETE = 38, DESC = 39, DESCENDING = 40, DESCRIBE = 41, + DETACH = 42, DICTIONARIES = 43, DICTIONARY = 44, DISK = 45, DISTINCT = 46, + DISTRIBUTED = 47, DROP = 48, ELSE = 49, END = 50, ENGINE = 51, EVENTS = 52, + EXISTS = 53, EXPLAIN = 54, EXPRESSION = 55, EXTRACT = 56, FETCHES = 57, + FINAL = 58, FIRST = 59, FLUSH = 60, FOR = 61, FORMAT = 62, FREEZE = 63, + FROM = 64, FULL = 65, FUNCTION = 66, GLOBAL = 67, GRANULARITY = 68, + GROUP = 69, HAVING = 70, HIERARCHICAL = 71, HOUR = 72, ID = 73, IF = 74, + ILIKE = 75, IN = 76, INDEX = 77, INF = 78, INJECTIVE = 79, INNER = 80, + INSERT = 81, INTERVAL = 82, INTO = 83, IS = 84, IS_OBJECT_ID = 85, JOIN = 86, + KEY = 87, KILL = 88, LAST = 89, LAYOUT = 90, LEADING = 91, LEFT = 92, + LIFETIME = 93, LIKE = 94, LIMIT = 95, LIVE = 96, LOCAL = 97, LOGS = 98, + MATERIALIZED = 99, MAX = 100, MERGES = 101, MIN = 102, MINUTE = 103, + MODIFY = 104, MONTH = 105, MOVE = 106, MUTATION = 107, NAN_SQL = 108, + NO = 109, NOT = 110, NULL_SQL = 111, NULLS = 112, OFFSET = 113, ON = 114, + OPTIMIZE = 115, OR = 116, ORDER = 117, OUTER = 118, OUTFILE = 119, PARTITION = 120, + POPULATE = 121, PREWHERE = 122, PRIMARY = 123, QUARTER = 124, RANGE = 125, + RELOAD = 126, REMOVE = 127, RENAME = 128, REPLACE = 129, REPLICA = 130, + REPLICATED = 131, RIGHT = 132, ROLLUP = 133, SAMPLE = 134, SECOND = 135, + SELECT = 136, SEMI = 137, SENDS = 138, SET = 139, SETTINGS = 140, SHOW = 141, + SOURCE = 142, START = 143, STOP = 144, SUBSTRING = 145, SYNC = 146, + SYNTAX = 147, SYSTEM = 148, TABLE = 149, TABLES = 150, TEMPORARY = 151, + TEST = 152, THEN = 153, TIES = 154, TIMEOUT = 155, TIMESTAMP = 156, + TO = 157, TOP = 158, TOTALS = 159, TRAILING = 160, TRIM = 161, TRUNCATE = 162, + TTL = 163, TYPE = 164, UNION = 165, UPDATE = 166, USE = 167, USING = 168, + UUID = 169, VALUES = 170, VIEW = 171, VOLUME = 172, WATCH = 173, WEEK = 174, + WHEN = 175, WHERE = 176, WITH = 177, YEAR = 178, JSON_FALSE = 179, JSON_TRUE = 180, + IDENTIFIER = 181, FLOATING_LITERAL = 182, OCTAL_LITERAL = 183, DECIMAL_LITERAL = 184, + HEXADECIMAL_LITERAL = 185, STRING_LITERAL = 186, ARROW = 187, ASTERISK = 188, + BACKQUOTE = 189, BACKSLASH = 190, COLON = 191, COMMA = 192, CONCAT = 193, + DASH = 194, DOT = 195, EQ_DOUBLE = 196, EQ_SINGLE = 197, GE = 198, GT = 199, + LBRACE = 200, LBRACKET = 201, LE = 202, LPAREN = 203, LT = 204, NOT_EQ = 205, + PERCENT = 206, PLUS = 207, QUERY = 208, QUOTE_DOUBLE = 209, QUOTE_SINGLE = 210, + RBRACE = 211, RBRACKET = 212, RPAREN = 213, SEMICOLON = 214, SLASH = 215, + UNDERSCORE = 216, MULTI_LINE_COMMENT = 217, SINGLE_LINE_COMMENT = 218, WHITESPACE = 219 }; ClickHouseLexer(antlr4::CharStream *input); - ~ClickHouseLexer(); + ~ClickHouseLexer() override; virtual std::string getGrammarFileName() const override; virtual const std::vector& getRuleNames() const override; diff --git a/src/Parsers/New/ClickHouseParser.h b/src/Parsers/New/ClickHouseParser.h index 11beadb182e..35e8d81d7f8 100644 --- a/src/Parsers/New/ClickHouseParser.h +++ b/src/Parsers/New/ClickHouseParser.h @@ -91,7 +91,7 @@ public: }; ClickHouseParser(antlr4::TokenStream *input); - ~ClickHouseParser(); + ~ClickHouseParser() override; virtual std::string getGrammarFileName() const override; virtual const antlr4::atn::ATN& getATN() const override { return _atn; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index dd65a67698d..c91fa3a86d0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -48,11 +48,11 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( , quorum(quorum_) , quorum_timeout_ms(quorum_timeout_ms_) , max_parts_per_block(max_parts_per_block_) + , is_attach(is_attach_) , quorum_parallel(quorum_parallel_) , deduplicate(deduplicate_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) , optimize_on_insert(optimize_on_insert_) - , is_attach(is_attach_) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 0d845f6b055..95119c97470 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -1,4 +1,4 @@ -2include +#include #include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4ebbecc42d6..44a034c28fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1304,7 +1304,7 @@ String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataP getSettings()->use_minimalistic_checksums_in_zookeeper); } -MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(const LogEntry& entry) const +MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(const LogEntry& entry) const { const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); const String& part_checksum = entry.part_checksum; @@ -1331,7 +1331,7 @@ MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(co auto single_disk_volume = std::make_shared("volume_" + part_name, getDiskForPart(part_name, detached_dir)); - MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); + MergeTreeData::MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) continue; // TODO if we can, here would be return {}; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 10d0734910a..d9f7398d7d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -237,6 +237,8 @@ private: using LogEntry = ReplicatedMergeTreeLogEntry; using LogEntryPtr = LogEntry::Ptr; + using MergeTreeData::MutableDataPartPtr; + zkutil::ZooKeeperPtr current_zookeeper; /// Use only the methods below. mutable std::mutex current_zookeeper_mutex; /// To recreate the session in the background thread. From de88a7ca94c5a9a82344004010cec3a07437fcc6 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 16 Feb 2021 18:36:30 +0300 Subject: [PATCH 009/266] Fixed the UB with maybe-nullptr member access --- src/Storages/StorageReplicatedMergeTree.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 44a034c28fa..e8659fff892 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1365,6 +1365,8 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) ReplicatedMergeTreeBlockOutputStream (*this, getInMemoryMetadataPtr() , 0, 0, 0, false, false, false) .writeExistingPart(part); + const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; + if (entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART || entry.type == LogEntry::MERGE_PARTS || @@ -1374,25 +1376,22 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) /// The part may be still in the PreCommitted -> Committed transition so we first search /// among PreCommitted parts to definitely find the desired part if it exists. DataPartPtr existing_part = getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted}); + if (!existing_part) existing_part = getActiveContainingPart(entry.new_part_name); - const String part_path = replica_path + "/parts/" + existing_part->name; - /// Even if the part is local, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there. - if (existing_part && getZooKeeper()->exists(part_path)) + if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name)) { - if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)) - { + if (!is_get_or_attach || entry.source_replica != replica_name) LOG_DEBUG(log, "Skipping action for part {} because part {} already exists.", entry.new_part_name, existing_part->name); - } + return true; } } - if ((entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) && - entry.source_replica == replica_name) + if (is_get_or_attach && entry.source_replica == replica_name) LOG_WARNING(log, "Part {} from own log doesn't exist.", entry.new_part_name); /// Perhaps we don't need this part, because during write with quorum, the quorum has failed From f1ef382cf9434ae80e3d544a5e5d622bf7d1a0ed Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 19 Feb 2021 03:24:53 +0300 Subject: [PATCH 010/266] Added part_checksum to Replicated...Entry serialization. --- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 5 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++-------- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 9a2fb79002c..04dcf0be074 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -53,7 +53,8 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const break; case ATTACH_PART: - out << "attach\n" << new_part_name; + out << "attach\n" << new_part_name << "\n" + << "part_checksum: " << part_checksum << "\n"; break; case MERGE_PARTS: @@ -188,7 +189,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) else if (type_str == "attach") { type = ATTACH_PART; - in >> new_part_name; + in >> new_part_name >> "\npart_checksum: " >> part_checksum >> "\n"; } else if (type_str == "merge") { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e8659fff892..11976fbe714 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1322,9 +1322,6 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo part_iter.partition_id != target_part.partition_id) continue; - // TODO Check if we can compare by entry.new_part_name ?== part_iter.getPartName(), - // mostly sure we can't, but if we can, all this thing would work faster. - const String& part_name = part_iter.getPartName(); const String part_to_path = detached_dir + part_name; @@ -1334,7 +1331,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo MergeTreeData::MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) - continue; // TODO if we can, here would be return {}; + continue; return iter_part_ptr; } @@ -1367,10 +1364,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; - if (entry.type == LogEntry::GET_PART || - entry.type == LogEntry::ATTACH_PART || - entry.type == LogEntry::MERGE_PARTS || - entry.type == LogEntry::MUTATE_PART) + if (is_get_or_attach || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) { /// If we already have this part or a part covering it, we do not need to do anything. /// The part may be still in the PreCommitted -> Committed transition so we first search From eb1826a5e3111fec4817c4ac3275e7cc0ac6ee2a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 19 Feb 2021 17:28:29 +0300 Subject: [PATCH 011/266] Trying to figure out why iteration doesn't work --- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 11976fbe714..8d9647362fc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1309,24 +1309,23 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); const String& part_checksum = entry.part_checksum; - MergeTreePartInfo part_iter; - const Poco::DirectoryIterator dir_end; - - const String detached_dir = "detached/"; + Poco::DirectoryIterator dir_end; for (const String& path : getDataPaths()) { - for (Poco::DirectoryIterator it{path + detached_dir}; it != dir_end; ++it) + for (Poco::DirectoryIterator it{path + "detached/"}; it != dir_end; ++it) { + MergeTreePartInfo part_iter; + if (!MergeTreePartInfo::tryParsePartName(it.name(), &part_iter, format_version) || part_iter.partition_id != target_part.partition_id) continue; const String& part_name = part_iter.getPartName(); - const String part_to_path = detached_dir + part_name; + const String part_to_path = "detached/" + part_name; auto single_disk_volume = std::make_shared("volume_" + part_name, - getDiskForPart(part_name, detached_dir)); + getDiskForPart(part_name, "detached/")); MergeTreeData::MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); From 08126030d69cb1771f7a6cb9a96962d347f89705 Mon Sep 17 00:00:00 2001 From: pingyu Date: Fri, 26 Feb 2021 00:44:01 +0800 Subject: [PATCH 012/266] wip #14893 --- .gitmodules | 3 +++ CMakeLists.txt | 1 + cmake/find/datasketches.cmake | 29 +++++++++++++++++++++++++++++ contrib/datasketches-cpp | 1 + src/CMakeLists.txt | 4 ++++ 5 files changed, 38 insertions(+) create mode 100644 cmake/find/datasketches.cmake create mode 160000 contrib/datasketches-cpp diff --git a/.gitmodules b/.gitmodules index 7a2c5600e65..b8e87e06758 100644 --- a/.gitmodules +++ b/.gitmodules @@ -221,3 +221,6 @@ [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git +[submodule "contrib/datasketches-cpp"] + path = contrib/datasketches-cpp + url = https://github.com/apache/datasketches-cpp.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 9002f1df140..b46b6e6afdd 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -504,6 +504,7 @@ include (cmake/find/msgpack.cmake) include (cmake/find/cassandra.cmake) include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) +include (cmake/find/datasketches.cmake) set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/datasketches.cmake b/cmake/find/datasketches.cmake new file mode 100644 index 00000000000..93ad9e7ed48 --- /dev/null +++ b/cmake/find/datasketches.cmake @@ -0,0 +1,29 @@ +option (ENABLE_DATASKETCHES "Enable DataSketches" ${ENABLE_LIBRARIES}) + +if (ENABLE_DATASKETCHES) + +option (USE_INTERNAL_DATASKETCHES_LIBRARY "Set to FALSE to use system DataSketches library instead of bundled" ${NOT_UNBUNDLED}) + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/theta/CMakeLists.txt") + if (USE_INTERNAL_DATASKETCHES_LIBRARY) + message(WARNING "submodule contrib/datasketches-cpp is missing. to fix try run: \n git submodule update --init --recursive") + endif() + set(MISSING_INTERNAL_DATASKETCHES_LIBRARY 1) + set(USE_INTERNAL_DATASKETCHES_LIBRARY 0) +endif() + +if (USE_INTERNAL_DATASKETCHES_LIBRARY) + set(DATASKETCHES_LIBRARY theta) + set(DATASKETCHES_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/common/include" "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/theta/include") +elseif (NOT MISSING_INTERNAL_DATASKETCHES_LIBRARY) + find_library(DATASKETCHES_LIBRARY theta) + find_path(DATASKETCHES_INCLUDE_DIR NAMES theta_sketch.hpp PATHS ${DATASKETCHES_INCLUDE_PATHS}) +endif() + +if (DATASKETCHES_LIBRARY AND DATASKETCHES_INCLUDE_DIR) + set(USE_DATASKETCHES 1) +endif() + +endif() + +message (STATUS "Using datasketches=${USE_DATASKETCHES}: ${DATASKETCHES_INCLUDE_DIR} : ${DATASKETCHES_LIBRARY}") \ No newline at end of file diff --git a/contrib/datasketches-cpp b/contrib/datasketches-cpp new file mode 160000 index 00000000000..c1a6f8edb49 --- /dev/null +++ b/contrib/datasketches-cpp @@ -0,0 +1 @@ +Subproject commit c1a6f8edb49699520f248d3d02019b87429b4241 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 215a13cce1a..5da0e3af890 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -456,6 +456,10 @@ if (USE_LIBPQXX) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${LIBPQXX_INCLUDE_DIR}) endif() +if (USE_DATASKETCHES) + target_include_directories (clickhouse_aggregate_functions SYSTEM BEFORE PRIVATE ${DATASKETCHES_INCLUDE_DIR}) +endif () + dbms_target_link_libraries(PRIVATE _boost_context) if (ENABLE_TESTS AND USE_GTEST) From b2c898f58c8166ce0d2b4441c6b427ec9eabed4e Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 25 Feb 2021 21:25:55 +0300 Subject: [PATCH 013/266] Adding the part when found --- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 28 ++++++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 3 +++ 3 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2f531bd8391..3b5e04feb0e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -198,8 +198,8 @@ public: * * Possible state transitions: * Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set - * Precommitted -> Outdated: we could not to add a part to active set and doing a rollback (for example it is duplicated part) - * Precommitted -> Committed: we successfully committed a part to active dataset + * Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part) + * Precommitted -> Committed: we successfully committed a part to active dataset * Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION * Outdated -> Deleting: a cleaner selected this part for deletion * Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8d9647362fc..0dde2c6dc9b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1339,6 +1339,12 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo return {}; } +void StorageReplicatedMergeTree::attachPart(MutableDataPartPtr& part) +{ + Transaction transaction(*this); + renameTempPartAndAdd(part, nullptr, &transaction); + checkPartChecksumsAndCommit(transaction, part); +} bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { @@ -1354,12 +1360,9 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } - /// Try to look in the detached/ folder first, if found, attach the part if (entry.type == LogEntry::ATTACH_PART) if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) - // no need to call checkAlterPartitionIsPossible as we already parsed the part name - ReplicatedMergeTreeBlockOutputStream (*this, getInMemoryMetadataPtr() , 0, 0, 0, false, false, false) - .writeExistingPart(part); + attachPart(part); const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; @@ -1458,7 +1461,8 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) if (replica_to_execute_merge) { - LOG_DEBUG(log, "Prefer fetching part {} from replica {} due execute_merges_on_single_replica_time_threshold", entry.new_part_name, replica_to_execute_merge.value()); + LOG_DEBUG(log, "Prefer fetching part {} from replica {} due execute_merges_on_single_replica_time_threshold", + entry.new_part_name, replica_to_execute_merge.value()); return false; } } @@ -1739,21 +1743,21 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) auto metadata_snapshot = getInMemoryMetadataPtr(); static std::atomic_uint total_fetches {0}; - if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) - { - throw Exception("Too many total fetches from replicas, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches.toString(), + + if (storage_settings_ptr->replicated_max_parallel_fetches && + total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) + throw Exception("Too many total fetches from replicas, maximum: " + + storage_settings_ptr->replicated_max_parallel_fetches.toString(), ErrorCodes::TOO_MANY_FETCHES); - } ++total_fetches; SCOPE_EXIT({--total_fetches;}); if (storage_settings_ptr->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table) - { - throw Exception("Too many fetches from replicas for table, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(), + throw Exception("Too many fetches from replicas for table, maximum: " + + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(), ErrorCodes::TOO_MANY_FETCHES); - } ++current_table_fetches; SCOPE_EXIT({--current_table_fetches;}); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d9f7398d7d5..b4c04078b56 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -410,6 +410,9 @@ private: /// returns nullptr if the part is corrupt or missing. MutableDataPartPtr attachPartHelperFoundValidPart(const LogEntry& entry) const; + /// Attach the part found in the detached/ folder + void attachPart(MutableDataPartPtr& part); + void executeDropRange(const LogEntry & entry); /// Do the merge or recommend to make the fetch instead of the merge From 2b3b335edae019231934508961b1bd30e25af454 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 25 Feb 2021 21:41:09 +0300 Subject: [PATCH 014/266] Exit fixes --- src/Storages/StorageReplicatedMergeTree.cpp | 18 ++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 3 --- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0dde2c6dc9b..555e97ad1de 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1339,13 +1339,6 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo return {}; } -void StorageReplicatedMergeTree::attachPart(MutableDataPartPtr& part) -{ - Transaction transaction(*this); - renameTempPartAndAdd(part, nullptr, &transaction); - checkPartChecksumsAndCommit(transaction, part); -} - bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { if (entry.type == LogEntry::DROP_RANGE) @@ -1361,8 +1354,17 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) } if (entry.type == LogEntry::ATTACH_PART) + { if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) - attachPart(part); + { + Transaction transaction(*this); + + if (renameTempPartAndAdd(part, nullptr, &transaction)) + checkPartChecksumsAndCommit(transaction, part); + + return true; + } + } const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index b4c04078b56..d9f7398d7d5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -410,9 +410,6 @@ private: /// returns nullptr if the part is corrupt or missing. MutableDataPartPtr attachPartHelperFoundValidPart(const LogEntry& entry) const; - /// Attach the part found in the detached/ folder - void attachPart(MutableDataPartPtr& part); - void executeDropRange(const LogEntry & entry); /// Do the merge or recommend to make the fetch instead of the merge From d331954f16a8d8cfbe0098e6969a4b51dcf7be5f Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 26 Feb 2021 01:57:20 +0300 Subject: [PATCH 015/266] DB engine --- docs/en/engines/database-engines/index.md | 4 + .../en/engines/database-engines/postgresql.md | 138 ++++++++++++++++++ 2 files changed, 142 insertions(+) create mode 100644 docs/en/engines/database-engines/postgresql.md diff --git a/docs/en/engines/database-engines/index.md b/docs/en/engines/database-engines/index.md index 2db11998483..b6892099378 100644 --- a/docs/en/engines/database-engines/index.md +++ b/docs/en/engines/database-engines/index.md @@ -18,4 +18,8 @@ You can also use the following database engines: - [Lazy](../../engines/database-engines/lazy.md) +- [Atomic](../../engines/database-engines/atomic.md) + +- [PostgreSQL](../../engines/database-engines/postgresql.md) + [Original article](https://clickhouse.tech/docs/en/database_engines/) diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md new file mode 100644 index 00000000000..078288d960d --- /dev/null +++ b/docs/en/engines/database-engines/postgresql.md @@ -0,0 +1,138 @@ +--- +toc_priority: 35 +toc_title: PostgreSQL +--- + +# PostgreSQL {#postgresql} + +Allows to connect to databases on a remote PostgreSQL server. Supports read and write operations (`SELECT` and `INSERT` queries) to exchange data between ClickHouse and PostgreSQL. + +Gives the real-time access to table list and table structure from remote PostgreSQL with the help of `SHOW TABLES` and `DESCRIBE TABLE` queries. + +Supports table structure modifications (`ALTER TABLE ... ADD|DROP COLUMN`). If `use_table_cache` parameter (see Engine Parameters below) it set to `1`, the table structure is cached and not checked for being modified, but can be updated with `DETACH` and `ATTACH` queries. + +## Creating a Database {#creating-a-database} + +``` sql +CREATE DATABASE test_database +ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cache`]); +``` + +**Engine Parameters** + +- `host:port` — PostgreSQL server address. +- `database` — Remote database name. +- `user` — PostgreSQL user. +- `password` — User password. +- `use_table_cache` — Defines if the database table structure is cached or not. Optional. Default value: `0`. + +## Data Types Support {#data_types-support} + +| PostgerSQL | ClickHouse | +|------------------|--------------------------------------------------------------| +| DATE | [Date](../../sql-reference/data-types/date.md) | +| TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) | +| REAL | [Float32](../../sql-reference/data-types/float.md) | +| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | +| DECIMAL, NUMERIC | [Decimal](../../sql-reference/data-types/decimal.md) | +| SMALLINT | [Int16](../../sql-reference/data-types/int-uint.md) | +| INTEGER | [Int32](../../sql-reference/data-types/int-uint.md) | +| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) | +| SERIAL | [UInt32](../../sql-reference/data-types/int-uint.md) | +| BIGSERIAL | [UInt64](../../sql-reference/data-types/int-uint.md) | +| TEXT, CHAR | [String](../../sql-reference/data-types/string.md) | +| INTEGER | Nullable([Int32](../../sql-reference/data-types/int-uint.md))| +| ARRAY | [Array](../../sql-reference/data-types/array.md) | + + +## Examples of Use {#examples-of-use} + +Database in ClickHouse, exchanging data with the PostgreSQL server: + +``` sql +CREATE DATABASE test_database +ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword', 1); +``` + +``` sql +SHOW DATABASES; +``` + +``` text +┌─name──────────┐ +│ default │ +│ test_database │ +│ system │ +└───────────────┘ +``` + +``` sql +SHOW TABLES FROM test_database; +``` + +``` text +┌─name───────┐ +│ test_table │ +└────────────┘ +``` + +Reading data from the PostgreSQL table: + +``` sql +SELECT * FROM test_database.test_table; +``` + +``` text +┌─id─┬─value─┐ +│ 1 │ 2 │ +└────┴───────┘ +``` + +Writing data to the PostgreSQL table: + +``` sql +INSERT INTO test_database.test_table VALUES (3,4); +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +Consider the table structure was modified in PostgreSQL: + +``` +postgre> ALTER TABLE test_table ADD COLUMN data Text +``` + +As the `use_table_cache` parameter was set to `1` when the database was created, the table structure in ClickHouse was cached and therefore not modified: + +``` sql +DESCRIBE TABLE test_database.test_table; +``` +``` text +┌─name───┬─type──────────────┐ +│ id │ Nullable(Integer) │ +│ value │ Nullable(Integer) │ +└────────┴───────────────────┘ +``` + +After detaching the table and attaching it again, the structure was updated: + +``` sql +DETACH TABLE test_database.test_table; +ATTACH TABLE test_database.test_table; +DESCRIBE TABLE test_database.test_table; +``` +``` text +┌─name───┬─type──────────────┐ +│ id │ Nullable(Integer) │ +│ value │ Nullable(Integer) │ +│ data │ Nullable(String) │ +└────────┴───────────────────┘ +``` + +[Original article](https://clickhouse.tech/docs/en/database_engines/postgresql/) From 64322d221272581394cdde475cca2e6776108004 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 27 Feb 2021 03:01:02 +0300 Subject: [PATCH 016/266] Table engine and function --- .../en/engines/database-engines/postgresql.md | 2 +- .../integrations/embedded-rocksdb.md | 2 +- .../table-engines/integrations/hdfs.md | 2 +- .../table-engines/integrations/index.md | 3 +- .../table-engines/integrations/jdbc.md | 2 +- .../table-engines/integrations/kafka.md | 2 +- .../table-engines/integrations/mongodb.md | 2 +- .../table-engines/integrations/mysql.md | 2 +- .../table-engines/integrations/odbc.md | 2 +- .../table-engines/integrations/postgresql.md | 77 +++++++++++++++++++ .../table-engines/integrations/rabbitmq.md | 2 +- .../engines/table-engines/integrations/s3.md | 2 +- .../table-functions/postgresql.md | 77 +++++++++++++++++++ 13 files changed, 166 insertions(+), 11 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/postgresql.md create mode 100644 docs/en/sql-reference/table-functions/postgresql.md diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md index 078288d960d..3171351352f 100644 --- a/docs/en/engines/database-engines/postgresql.md +++ b/docs/en/engines/database-engines/postgresql.md @@ -9,7 +9,7 @@ Allows to connect to databases on a remote PostgreSQL server. Supports read and Gives the real-time access to table list and table structure from remote PostgreSQL with the help of `SHOW TABLES` and `DESCRIBE TABLE` queries. -Supports table structure modifications (`ALTER TABLE ... ADD|DROP COLUMN`). If `use_table_cache` parameter (see Engine Parameters below) it set to `1`, the table structure is cached and not checked for being modified, but can be updated with `DETACH` and `ATTACH` queries. +Supports table structure modifications (`ALTER TABLE ... ADD|DROP COLUMN`). If `use_table_cache` parameter (see the Engine Parameters below) it set to `1`, the table structure is cached and not checked for being modified, but can be updated with `DETACH` and `ATTACH` queries. ## Creating a Database {#creating-a-database} diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 6e864751cc3..45126407a75 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -1,5 +1,5 @@ --- -toc_priority: 6 +toc_priority: 9 toc_title: EmbeddedRocksDB --- diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 5c36e3f1c21..92754fd32a1 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -1,5 +1,5 @@ --- -toc_priority: 4 +toc_priority: 6 toc_title: HDFS --- diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index 288c9c3cd56..eb1c5411e18 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -1,6 +1,6 @@ --- toc_folder_title: Integrations -toc_priority: 30 +toc_priority: 1 --- # Table Engines for Integrations {#table-engines-for-integrations} @@ -18,3 +18,4 @@ List of supported integrations: - [Kafka](../../../engines/table-engines/integrations/kafka.md) - [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md) - [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) +- [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md) diff --git a/docs/en/engines/table-engines/integrations/jdbc.md b/docs/en/engines/table-engines/integrations/jdbc.md index 2144be9f1e3..ad6f58c386d 100644 --- a/docs/en/engines/table-engines/integrations/jdbc.md +++ b/docs/en/engines/table-engines/integrations/jdbc.md @@ -1,5 +1,5 @@ --- -toc_priority: 2 +toc_priority: 3 toc_title: JDBC --- diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index c519d6bb136..9da1a51f26f 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -1,5 +1,5 @@ --- -toc_priority: 5 +toc_priority: 8 toc_title: Kafka --- diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index e648a13b5e0..eb3d5060b01 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -1,5 +1,5 @@ --- -toc_priority: 7 +toc_priority: 5 toc_title: MongoDB --- diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index 2cb1facce91..ee712ccf871 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -1,5 +1,5 @@ --- -toc_priority: 3 +toc_priority: 4 toc_title: MySQL --- diff --git a/docs/en/engines/table-engines/integrations/odbc.md b/docs/en/engines/table-engines/integrations/odbc.md index fffc125b0ff..f3689da6af8 100644 --- a/docs/en/engines/table-engines/integrations/odbc.md +++ b/docs/en/engines/table-engines/integrations/odbc.md @@ -1,5 +1,5 @@ --- -toc_priority: 1 +toc_priority: 2 toc_title: ODBC --- diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md new file mode 100644 index 00000000000..b64b22220ff --- /dev/null +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -0,0 +1,77 @@ +--- +toc_priority: 11 +toc_title: PostgreSQL +--- + +#PostgreSQL {#postgresql} + +The PostgreSQL engine allows to perform `SELECT` and `INSERT` queries on data that is stored on a remote PostgreSQL server. + +## Creating a Table {#creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + ... +) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); +``` + +See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. + +**Engine Parameters** + +- `host:port` — MySQL server address. + +- `database` — Remote database name. + +- `table` — Remote table name. + +- `user` — MySQL user. + +- `password` — User password. + +## Usage Example {#usage-example} + +Consider the table in ClickHouse, retrieving data from the PostgreSQL table: + +``` sql +CREATE TABLE test_table +( + `int_id` Int32, + 'value' Int32 +) +ENGINE = PostgreSQL('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` sql +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` + +Inserting data from ClickHouse into the PosegreSQL table: + +``` sql +INSERT INTO test_database.test_table VALUES (3,4); +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +## See Also {#see-also} + +- [The 'postgresql' table function](../../../sql-reference/table-functions/postgresql.md) +- [Using PostgreSQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#example-of-connecting-postgresql) + +[Original article](https://clickhouse.tech/docs/en/operations/table_engines/postgresql/) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index 4a0550275ca..3c9a96cff29 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -1,5 +1,5 @@ --- -toc_priority: 6 +toc_priority: 10 toc_title: RabbitMQ --- diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 5858a0803e6..47c63db9e09 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -1,5 +1,5 @@ --- -toc_priority: 4 +toc_priority: 7 toc_title: S3 --- diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md new file mode 100644 index 00000000000..65a04bb8f60 --- /dev/null +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -0,0 +1,77 @@ +--- +toc_priority: 54 +toc_title: postgresql +--- + +# postgresql {#postgresql} + +Allows `SELECT` and `INSERT` queries to be performed on data that is stored on a remote PostgreSQL server. + +**Syntax** + +``` sql +postgresql('host:port', 'database', 'table', 'user', 'password') +``` + +**Arguments** + +- `host:port` — PostgreSQL server address. + +- `database` — Remote database name. + +- `table` — Remote table name. + +- `user` — PostgreSQL user. + +- `password` — User password. + +**Returned Value** + +A table object with the same columns as the original PostgreSQL table. + +!!! info "Note" + In the `INSERT` query to distinguish table function `postgresql(...)` from table name with column names list, you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. + +**Examples** + +Consider the table in PostgreSQL: + +``` sql +postgre> CREATE TABLE IF NOT EXISTS test_table (a integer, b text, c integer) +postgre> INSERT INTO test_table (a, b, c) VALUES (1, 2, 3), (4, 5, 6) +``` + +Selecting data from ClickHouse: + +``` sql +SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` text +┌─a─┬─b─┬─c─┐ +│ 1 │ 2 │ 3 │ +│ 4 │ 5 │ 6 │ +└───┴───┴───┘ +``` + +Inserting into PostgreSQL from ClickHouse: + +```sql +INSERT INTO FUNCTION postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword') (a, b, c) VALUES (7, 8, 9); +SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` text +┌─a─┬─b─┬─c─┐ +│ 1 │ 2 │ 3 │ +│ 4 │ 5 │ 6 │ +│ 7 │ 8 │ 9 │ +└───┴───┴───┘ +``` + +**See Also** + +- [The ‘MySQL’ table engine](../../engines/table-engines/integrations/postgresql.md) +- [Using MySQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#example-of-connecting-postgresql) + +[Original article](https://clickhouse.tech/docs/en/sql-reference/table_functions/postgresql/) From c11824b156caa366bebb124738c4e40bb9572984 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 27 Feb 2021 03:29:56 +0300 Subject: [PATCH 017/266] Dictionary --- .../table-engines/integrations/postgresql.md | 2 +- .../external-dicts-dict-sources.md | 50 +++++++++++++++++++ .../table-functions/postgresql.md | 6 +-- 3 files changed, 52 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index b64b22220ff..48e183b6d32 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -72,6 +72,6 @@ SELECT * FROM test_database.test_table; ## See Also {#see-also} - [The 'postgresql' table function](../../../sql-reference/table-functions/postgresql.md) -- [Using PostgreSQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#example-of-connecting-postgresql) +- [Using PostgreSQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) [Original article](https://clickhouse.tech/docs/en/operations/table_engines/postgresql/) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 7cd26a9dffb..65ad141979d 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -68,6 +68,8 @@ Types of sources (`source_type`): - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) - [Redis](#dicts-external_dicts_dict_sources-redis) + - [Cassandra](#dicts-external_dicts_dict_sources-cassandra) + - [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql) ## Local File {#dicts-external_dicts_dict_sources-local_file} @@ -673,4 +675,52 @@ Default value is 1 (the first key column is a partition key and other key column - `where` – Optional selection criteria. - `max_threads` – The maximum number of threads to use for loading data from multiple partitions in compose key dictionaries. +### PostgreSQL {#dicts-external_dicts_dict_sources-postgresql} + +Example of settings: + +``` xml + + + clickhouse + localhost + 5432 + postgres + mysecretpassword + test_table
+ SELECT value FROM test_table WHERE id = 0 +
+ + + + + + + id + UInt32 + + + id + UInt32 + + + + value + UInt32 + + + +1 +``` + +Setting fields: + +- `db` — Remote database name. +- `host` — The PostgreSQL host. +- `port` – The port on the PostgreSQL server. If not specified, default port is used. +- `user` — PostgreSQL user. +- `password` — User password. +- `table` — Remote table name. +- `invalidate_query` — Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). + [Original article](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_sources/) diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 65a04bb8f60..ff1377aea2b 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -16,13 +16,9 @@ postgresql('host:port', 'database', 'table', 'user', 'password') **Arguments** - `host:port` — PostgreSQL server address. - - `database` — Remote database name. - - `table` — Remote table name. - - `user` — PostgreSQL user. - - `password` — User password. **Returned Value** @@ -72,6 +68,6 @@ SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postg **See Also** - [The ‘MySQL’ table engine](../../engines/table-engines/integrations/postgresql.md) -- [Using MySQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#example-of-connecting-postgresql) +- [Using MySQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) [Original article](https://clickhouse.tech/docs/en/sql-reference/table_functions/postgresql/) From b6b98bd09a76422e61755679ca482b2f565bb8b9 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 27 Feb 2021 03:42:45 +0300 Subject: [PATCH 018/266] Updates for Readme and master fixes --- docs/README.md | 7 +++++-- docs/en/sql-reference/functions/hash-functions.md | 4 ++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/README.md b/docs/README.md index 8b3066501bf..a05f7df8968 100644 --- a/docs/README.md +++ b/docs/README.md @@ -122,7 +122,7 @@ Contribute all new information in English language. Other languages are translat | Cell C1 | Cell C2 | Cell C3 | ``` - +{## ### Adding a New File @@ -134,7 +134,7 @@ When adding a new file: $ cd /ClickHouse/clone/directory/docs $ ln -sr en/new/file.md lang/new/file.md ``` - +##} ### Adding a New Language @@ -195,8 +195,11 @@ Templates: - [Function](_description_templates/template-function.md) - [Setting](_description_templates/template-setting.md) +- [Server Setting](_description_templates/template-server-setting.md) - [Database or Table engine](_description_templates/template-engine.md) - [System table](_description_templates/template-system-table.md) +- [Data type](_description_templates/data-type.md) +- [Statement](_description_templates/statement.md) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 6bf1bebabaa..465ad01527f 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -9,7 +9,7 @@ Hash functions can be used for the deterministic pseudo-random shuffling of elem ## halfMD5 {#hash-functions-halfmd5} -[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order. +[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order. ``` sql halfMD5(par1, ...) @@ -54,7 +54,7 @@ sipHash64(par1,...) This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function. -Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm: +Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm: 1. After hashing all the input parameters, the function gets the array of hashes. 2. Function takes the first and the second elements and calculates a hash for the array of them. From 106772af33321dd15a63da51c7b7ac86d7aca09c Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 27 Feb 2021 03:59:03 +0300 Subject: [PATCH 019/266] Fixes --- docs/en/engines/database-engines/postgresql.md | 2 +- docs/en/sql-reference/table-functions/postgresql.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md index 3171351352f..d21eb68bbdb 100644 --- a/docs/en/engines/database-engines/postgresql.md +++ b/docs/en/engines/database-engines/postgresql.md @@ -104,7 +104,7 @@ SELECT * FROM test_database.test_table; Consider the table structure was modified in PostgreSQL: -``` +``` sql postgre> ALTER TABLE test_table ADD COLUMN data Text ``` diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index ff1377aea2b..60d813a08e7 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -67,7 +67,7 @@ SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postg **See Also** -- [The ‘MySQL’ table engine](../../engines/table-engines/integrations/postgresql.md) -- [Using MySQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [The PostgreSQL table engine](../../engines/table-engines/integrations/postgresql.md) +- [Using PostgreSQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) [Original article](https://clickhouse.tech/docs/en/sql-reference/table_functions/postgresql/) From 720bef8ae207f65012c3c4405ade2d7c41eb9aac Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 27 Feb 2021 04:14:56 +0300 Subject: [PATCH 020/266] Links in index.md files --- docs/en/engines/table-engines/index.md | 16 +++++++----- .../en/sql-reference/table-functions/index.md | 25 ++++++++++--------- 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index 546557beb57..303c0697cb8 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -47,12 +47,16 @@ Engines for communicating with other data storage and processing systems. Engines in the family: -- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) -- [MySQL](../../engines/table-engines/integrations/mysql.md#mysql) -- [ODBC](../../engines/table-engines/integrations/odbc.md#table-engine-odbc) -- [JDBC](../../engines/table-engines/integrations/jdbc.md#table-engine-jdbc) -- [HDFS](../../engines/table-engines/integrations/hdfs.md#hdfs) -- [S3](../../engines/table-engines/integrations/s3.md#table_engines-s3) +- [ODBC](../../engines/table-engines/integrations/odbc.md) +- [JDBC](../../engines/table-engines/integrations/jdbc.md) +- [MySQL](../../engines/table-engines/integrations/mysql.md) +- [MongoDB](../../engines/table-engines/integrations/mongodb.md) +- [HDFS](../../engines/table-engines/integrations/hdfs.md) +- [S3](../../engines/table-engines/integrations/s3.md) +- [Kafka](../../engines/table-engines/integrations/kafka.md) +- [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) +- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) +- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) ### Special Engines {#special-engines} diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index 691687dea25..38d98f11402 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -21,17 +21,18 @@ You can use table functions in: !!! warning "Warning" You can’t use table functions if the [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. -| Function | Description | -|-----------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------| -| [file](../../sql-reference/table-functions/file.md) | Creates a [File](../../engines/table-engines/special/file.md)-engine table. | -| [merge](../../sql-reference/table-functions/merge.md) | Creates a [Merge](../../engines/table-engines/special/merge.md)-engine table. | -| [numbers](../../sql-reference/table-functions/numbers.md) | Creates a table with a single column filled with integer numbers. | -| [remote](../../sql-reference/table-functions/remote.md) | Allows you to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md)-engine table. | -| [url](../../sql-reference/table-functions/url.md) | Creates a [Url](../../engines/table-engines/special/url.md)-engine table. | -| [mysql](../../sql-reference/table-functions/mysql.md) | Creates a [MySQL](../../engines/table-engines/integrations/mysql.md)-engine table. | -| [jdbc](../../sql-reference/table-functions/jdbc.md) | Creates a [JDBC](../../engines/table-engines/integrations/jdbc.md)-engine table. | -| [odbc](../../sql-reference/table-functions/odbc.md) | Creates a [ODBC](../../engines/table-engines/integrations/odbc.md)-engine table. | -| [hdfs](../../sql-reference/table-functions/hdfs.md) | Creates a [HDFS](../../engines/table-engines/integrations/hdfs.md)-engine table. | -| [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | +| Function | Description | +|------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------| +| [file](../../sql-reference/table-functions/file.md) | Creates a [File](../../engines/table-engines/special/file.md)-engine table. | +| [merge](../../sql-reference/table-functions/merge.md) | Creates a [Merge](../../engines/table-engines/special/merge.md)-engine table. | +| [numbers](../../sql-reference/table-functions/numbers.md) | Creates a table with a single column filled with integer numbers. | +| [remote](../../sql-reference/table-functions/remote.md) | Allows you to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md)-engine table. | +| [url](../../sql-reference/table-functions/url.md) | Creates a [Url](../../engines/table-engines/special/url.md)-engine table. | +| [mysql](../../sql-reference/table-functions/mysql.md) | Creates a [MySQL](../../engines/table-engines/integrations/mysql.md)-engine table. | +| [jdbc](../../sql-reference/table-functions/jdbc.md) | Creates a [JDBC](../../engines/table-engines/integrations/jdbc.md)-engine table. | +| [odbc](../../sql-reference/table-functions/odbc.md) | Creates a [ODBC](../../engines/table-engines/integrations/odbc.md)-engine table. | +| [hdfs](../../sql-reference/table-functions/hdfs.md) | Creates a [HDFS](../../engines/table-engines/integrations/hdfs.md)-engine table. | +| [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | +| [postgresql](../../sql-reference/table-functions/postgresql.md) | Creates a [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)-engine table. | [Original article](https://clickhouse.tech/docs/en/query_language/table_functions/) From 5281314ac0b5d68ae87004cc429c5e6b4c0dc242 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 1 Mar 2021 16:42:31 +0300 Subject: [PATCH 021/266] Finished the test draft for ATTACH PARTITION, Extracted the part data corruption function into the helper. --- .../helpers/corrupt_part_data_on_disk.py | 7 ++ .../configs/remote_servers.xml | 21 ++++ .../test_attach_without_fetching/test.py | 97 ++++++++++++++----- .../test_broken_part_during_merge/test.py | 10 +- 4 files changed, 103 insertions(+), 32 deletions(-) create mode 100644 tests/integration/helpers/corrupt_part_data_on_disk.py create mode 100644 tests/integration/test_attach_without_fetching/configs/remote_servers.xml diff --git a/tests/integration/helpers/corrupt_part_data_on_disk.py b/tests/integration/helpers/corrupt_part_data_on_disk.py new file mode 100644 index 00000000000..c60a55d12d3 --- /dev/null +++ b/tests/integration/helpers/corrupt_part_data_on_disk.py @@ -0,0 +1,7 @@ +def corrupt_part_data_on_disk(node, table, part_name, is_detached=False): + parts_table = "system.detached_parts" is is_detached else "system.parts" + part_path = node.query( + "SELECT path FROM " + parts_table + " WHERE table = '{}' and name = '{}'".format(table, part_name)).strip() + node.exec_in_container(['bash', '-c', + 'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format( + p=part_path)], privileged=True) diff --git a/tests/integration/test_attach_without_fetching/configs/remote_servers.xml b/tests/integration/test_attach_without_fetching/configs/remote_servers.xml new file mode 100644 index 00000000000..7978f921b2e --- /dev/null +++ b/tests/integration/test_attach_without_fetching/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + true + + node_1_1 + 9000 + + + node_1_2 + 9000 + + + node_1_3 + 9000 + + + + + diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index d712cf5d2e3..a5e759c7fd6 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -1,49 +1,98 @@ +import time import pytest + from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseKiller +from helpers.test_tools import assert_eq_with_retry +from helpers.network import PartitionManager +from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk + +def fill_node(node): + node.query( + ''' + CREATE TABLE test(n UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', '{replica}') + ORDER BY n PARTITION BY n % 10; + '''.format(replica=node.name)) cluster = ClickHouseCluster(__file__) +configs =["configs/remote_servers.xml"] -node1 = cluster.add_instance('node1') - +node_1 = cluster.add_instance('replica1', with_zookeeper=True, main_configs=configs) +node_2 = cluster.add_instance('replica2', with_zookeeper=True, main_configs=configs) +node_3 = cluster.add_instance('replica3', with_zookeeper=True, main_configs=configs) @pytest.fixture(scope="module") def start_cluster(): try: cluster.start() - + fill_node(node_1) + fill_node(node_2) + # the third node is filled after the DETACH query yield cluster + + except Exception as ex: + print(ex) + finally: cluster.shutdown() +def check_data(nodes, detached_parts): + for node in nodes: + for i in range(10): + assert node.query("SELECT count() FROM test WHERE key % 10 == " + str(i)) == + "0\n" if i in detached_parts else "10\n" -# Check that ALTER TABLE ATTACH PARTITION does not fetch data from other replicas if it's present in the -# detached/ folder + assert node.query("SELECT count() FROM system.parts WHERE table='test'") == + str(10 - len(detached_parts)) + "\n" + + assert node.query("SELECT count() FROM system.detached_parts WHERE table='test'") == + str(len(detached_parts)) + "\n" + +# 1. Check that ALTER TABLE ATTACH PARTITION does not fetch data from other replicas if it's present in the +# detached/ folder. +# 2. Check that ALTER TABLE ATTACH PARTITION downloads the data from other replicas if the detached/ folder +# does not contain the part with the correct checksums. def test_attach_without_fetching(start_cluster): - node1.query( - "CREATE TABLE test (date Date, key Int32, value String) Engine=MergeTree ORDER BY key PARTITION by date") + # 0. Insert data on two replicas + node_1.query("INSERT INTO test SELECT * FROM numbers(100)") - node1.query("INSERT INTO test SELECT toDate('2019-10-01'), number, toString(number) FROM numbers(100)") + check_data([node_1, node_2], detached_parts=[]) - assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n" + # 1. Detach the first three partition on the replicas - node1.query("ALTER TABLE test DETACH PARTITION '2019-10-01'") + # This part will be fetched from other replicas as it would be missing in the detached/ folder + node_1.query("ALTER TABLE test DETACH PARTITION '0_0_0_0'") + # This part will be fetched from other replicas as the checksums won't match (we'll manually break the data). + node_1.query("ALTER TABLE test DETACH PARTITION '1_0_0_0'") + # This part will be copied locally and attached without fetch + node_1.query("ALTER TABLE test DETACH PARTITION '2_0_0_0'") - assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "0\n" - assert node1.query("SELECT COUNT() FROM test") == "0\n" + check_data([node_1, node_2], detached_parts=[0, 1, 2]) - # Break the network in the partition manager - # The data is not removed from detached/ so it's ok + # 2. Create the third replica + fill_node(node_3) - # to be sure output not empty - node1.exec_in_container( - ['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" | grep -e ".*" '], - privileged=True, user='root') + # 3. Attach the first partition and check if it has been fetched correctly + node_3.query("ALTER TABLE test ATTACH PARTITION '0_0_0_0'") + check_data([node_1, node_2, node_3], detached_parts=[1, 2]) - node1.exec_in_container( - ['bash', '-c', 'find /var/lib/clickhouse/data/default/test/detached -name "checksums.txt" -delete'], - privileged=True, user='root') + # 4. Fetch the second partition to the third replica, break the data to corrupt the checksums, + # attach it and check if it also was fetched correctly. + node_3.query("ALTER TABLE test FETCH PARTITION '1_0_0_0' FROM '/clickhouse/tables/test'") + corrupt_part_data_on_disk(node_3, 'test', '1_0_0_0', is_detached=True) + node_3.query("ALTER TABLE test ATTACH PARTITION '1_0_0_0'") - node1.query("ALTER TABLE test ATTACH PARTITION '2019-10-01'") + check_data([node_1, node_2, node_3], detached_parts=[2]) - assert node1.query("SELECT COUNT() FROM test WHERE key % 10 == 0") == "10\n" - assert node1.query("SELECT COUNT() FROM test") == "100\n" + # 5. Fetch the third partition to the third replica, break the network as so the replica won't be able to + # download the data, attach the partition (and check it has been attached from the local data) + node_3.query("ALTER TABLE test FETCH PARTITION '2_0_0_0' FROM '/clickhouse/tables/test'") + + with PartitionManager() as pm: + pm.partition_instances(node_1, node_3) + pm.partition_instances(node_2, node_3) + + node_3.query("ALTER TABLE test ATTACH PARTITION '2_0_0_0'") + + check_data([node_1, node_2, node_3], detached_parts=[]) diff --git a/tests/integration/test_broken_part_during_merge/test.py b/tests/integration/test_broken_part_during_merge/test.py index 33719166f4a..910dbc1d1a9 100644 --- a/tests/integration/test_broken_part_during_merge/test.py +++ b/tests/integration/test_broken_part_during_merge/test.py @@ -3,6 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from multiprocessing.dummy import Pool from helpers.network import PartitionManager +from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk import time cluster = ClickHouseCluster(__file__) @@ -25,13 +26,6 @@ def started_cluster(): finally: cluster.shutdown() -def corrupt_data_part_on_disk(node, table, part_name): - part_path = node.query( - "SELECT path FROM system.parts WHERE table = '{}' and name = '{}'".format(table, part_name)).strip() - node.exec_in_container(['bash', '-c', - 'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format( - p=part_path)], privileged=True) - def test_merge_and_part_corruption(started_cluster): node1.query("SYSTEM STOP REPLICATION QUEUES replicated_mt") @@ -43,7 +37,7 @@ def test_merge_and_part_corruption(started_cluster): # Need to corrupt "border part" (left or right). If we will corrupt something in the middle # clickhouse will not consider merge as broken, because we have parts with the same min and max # block numbers. - corrupt_data_part_on_disk(node1, 'replicated_mt', 'all_3_3_0') + corrupt_part_data_on_disk(node1, 'replicated_mt', 'all_3_3_0') with Pool(1) as p: def optimize_with_delay(x): From f088dd445dd37729a36f2a567e788560c4cf44e1 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 1 Mar 2021 18:12:02 +0300 Subject: [PATCH 022/266] Extended the test to check both the ALTER PARTITION and PART Added some notes about the SYSTEM SYNC REPLICA and ALTER ... DROP / ATTACH. --- .../statements/alter/partition.md | 14 +++- docs/en/sql-reference/statements/system.md | 2 + .../helpers/corrupt_part_data_on_disk.py | 11 ++- .../test_attach_without_fetching/test.py | 82 ++++++++++--------- 4 files changed, 65 insertions(+), 44 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 42396223b86..9efaafd4d54 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -40,7 +40,7 @@ Read about setting the partition expression in a section [How to specify the par After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it. -This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replica. +This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replicas (as multiple leaders are allowed). ## DROP PARTITION\|PART {#alter_drop-partition} @@ -61,6 +61,8 @@ ALTER TABLE mt DROP PARTITION '2020-11-21'; ALTER TABLE mt DROP PART 'all_4_4_0'; ``` +Note: the command does NOT throw an exception on an invalid `partition_expr`. + ## DROP DETACHED PARTITION\|PART {#alter_drop-detached} ``` sql @@ -70,6 +72,8 @@ ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr Removes the specified part or all parts of the specified partition from `detached`. Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Note: the command does NOT throw an exception on an invalid `partition_expr`. + ## ATTACH PARTITION\|PART {#alter_attach-partition} ``` sql @@ -85,7 +89,11 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). -This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. All other replicas download the data from the replica-initiator. +This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. + +If the non-initiator replica, receiving the attach command, finds the part with the correct checksums in its own `detached` folder, +it attaches the data without fetching it from other replicas. +If there is no part with the correct checksums, the data is downloaded from the replica-initiator. So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas. @@ -95,7 +103,7 @@ So you can put data to the `detached` directory on one replica, and use the `ALT ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 ``` -This query copies the data partition from the `table1` to `table2` adds data to exsisting in the `table2`. Note that data won’t be deleted from `table1`. +This query copies the data partition from the `table1` to `table2`. Note that data won’t be deleted from `table1`. For the query to run successfully, the following conditions must be met: diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index bb279703cc2..22e018d6ec9 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -264,6 +264,8 @@ Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name ``` +Note that after running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from other replicas. + ### RESTART REPLICA {#query_language-system-restart-replica} Provides possibility to reinitialize Zookeeper sessions state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed diff --git a/tests/integration/helpers/corrupt_part_data_on_disk.py b/tests/integration/helpers/corrupt_part_data_on_disk.py index c60a55d12d3..63bd2531063 100644 --- a/tests/integration/helpers/corrupt_part_data_on_disk.py +++ b/tests/integration/helpers/corrupt_part_data_on_disk.py @@ -1,7 +1,10 @@ -def corrupt_part_data_on_disk(node, table, part_name, is_detached=False): - parts_table = "system.detached_parts" is is_detached else "system.parts" - part_path = node.query( - "SELECT path FROM " + parts_table + " WHERE table = '{}' and name = '{}'".format(table, part_name)).strip() +def corrupt_part_data_on_disk(node, table, part_name): + part_path = node.query("SELECT path FROM system.parts WHERE table = '{}' and name = '{}'" + .format(table, part_name)).strip() + + corrupt_part_data_by_path(node, part_path) + +def corrupt_part_data_by_path(node, part_path): node.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format( p=part_path)], privileged=True) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index a5e759c7fd6..2f52d3e9846 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -2,10 +2,9 @@ import time import pytest from helpers.cluster import ClickHouseCluster -from helpers.cluster import ClickHouseKiller from helpers.test_tools import assert_eq_with_retry from helpers.network import PartitionManager -from helpers.corrupt_part_data_on_disk import corrupt_part_data_on_disk +from helpers.corrupt_part_data_on_disk import corrupt_part_data_by_path def fill_node(node): node.query( @@ -39,60 +38,69 @@ def start_cluster(): def check_data(nodes, detached_parts): for node in nodes: + node.query("SYSTEM SYNC REPLICA test") + + print("Checking data integrity for", node.name, "; parts missing:", detached_parts) + for i in range(10): - assert node.query("SELECT count() FROM test WHERE key % 10 == " + str(i)) == + assert node.query("SELECT count() FROM test WHERE n % 10 == " + str(i)) == \ "0\n" if i in detached_parts else "10\n" - assert node.query("SELECT count() FROM system.parts WHERE table='test'") == + print(node.query("SELECT partition_id, name FROM system.parts WHERE table='test'")) + + assert node.query("SELECT count() FROM system.parts WHERE table='test'") == \ str(10 - len(detached_parts)) + "\n" - assert node.query("SELECT count() FROM system.detached_parts WHERE table='test'") == - str(len(detached_parts)) + "\n" + # We don't check for system.detached_parts count = len(detached parts) as the newly downloaded data + # is not removed from detached_parts (so this is wrong) -# 1. Check that ALTER TABLE ATTACH PARTITION does not fetch data from other replicas if it's present in the + res: str = node.query("SELECT * FROM test ORDER BY n") + + for other in nodes: + if other != node: + assert_eq_with_retry(other, "SELECT * FROM test ORDER BY n", res) + + +# 1. Check that ALTER TABLE ATTACH PART|PARTITION does not fetch data from other replicas if it's present in the # detached/ folder. -# 2. Check that ALTER TABLE ATTACH PARTITION downloads the data from other replicas if the detached/ folder +# 2. Check that ALTER TABLE ATTACH PART|PARTITION downloads the data from other replicas if the detached/ folder # does not contain the part with the correct checksums. def test_attach_without_fetching(start_cluster): + # Note here requests are used for both PARTITION and PART. This is done for better test diversity. + # The partition and part are used interchangeably which is not true in most cases. # 0. Insert data on two replicas node_1.query("INSERT INTO test SELECT * FROM numbers(100)") check_data([node_1, node_2], detached_parts=[]) - # 1. Detach the first three partition on the replicas + path_to_detached_part_1 = node_1.query("SELECT path from system.parts WHERE table='test' AND name='1_0_0_0'") \ + .strip() + path_to_detached_part_1 = path_to_detached_part_1.split('/')[:-1] + path_to_detached_part_1 = path_to_detached_part_1[:-1] + ["detached", path_to_detached_part_1[-1]] + path_to_detached_part_1 = "/".join(path_to_detached_part_1) - # This part will be fetched from other replicas as it would be missing in the detached/ folder - node_1.query("ALTER TABLE test DETACH PARTITION '0_0_0_0'") - # This part will be fetched from other replicas as the checksums won't match (we'll manually break the data). - node_1.query("ALTER TABLE test DETACH PARTITION '1_0_0_0'") - # This part will be copied locally and attached without fetch - node_1.query("ALTER TABLE test DETACH PARTITION '2_0_0_0'") + # 1. Detach the first two partitions/parts on the replicas + # This part will be fetched from other replicas as it would be missing in the detached/ folder and + # also attached locally. + node_1.query("ALTER TABLE test DETACH PART '0_0_0_0'") + # This partition will be just fetched from other replicas as the checksums won't match + # (we'll manually break the data). + node_1.query("ALTER TABLE test DETACH PARTITION 1") - check_data([node_1, node_2], detached_parts=[0, 1, 2]) + check_data([node_1, node_2], detached_parts=[0, 1]) # 2. Create the third replica fill_node(node_3) - # 3. Attach the first partition and check if it has been fetched correctly - node_3.query("ALTER TABLE test ATTACH PARTITION '0_0_0_0'") - check_data([node_1, node_2, node_3], detached_parts=[1, 2]) - - # 4. Fetch the second partition to the third replica, break the data to corrupt the checksums, - # attach it and check if it also was fetched correctly. - node_3.query("ALTER TABLE test FETCH PARTITION '1_0_0_0' FROM '/clickhouse/tables/test'") - corrupt_part_data_on_disk(node_3, 'test', '1_0_0_0', is_detached=True) - node_3.query("ALTER TABLE test ATTACH PARTITION '1_0_0_0'") - - check_data([node_1, node_2, node_3], detached_parts=[2]) - - # 5. Fetch the third partition to the third replica, break the network as so the replica won't be able to - # download the data, attach the partition (and check it has been attached from the local data) - node_3.query("ALTER TABLE test FETCH PARTITION '2_0_0_0' FROM '/clickhouse/tables/test'") - - with PartitionManager() as pm: - pm.partition_instances(node_1, node_3) - pm.partition_instances(node_2, node_3) - - node_3.query("ALTER TABLE test ATTACH PARTITION '2_0_0_0'") + # 3. Attach the first part and check if it has been fetched correctly. + # Replica 3 should download the data from replica 2 as there is no local data. + # Replica 1 should attach the local data from detached/. + node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") + check_data([node_1, node_2, node_3], detached_parts=[1]) + # 4. Break the part data on the second node to corrupt the checksums. + # Replica 3 should download the data from replica 1 as there is no local data. + # Replica 2 should also download the data from 1 as the checksums won't match. + corrupt_part_data_by_path(node_2, path_to_detached_part_1) + node_1.query("ALTER TABLE test ATTACH PARTITION 1") check_data([node_1, node_2, node_3], detached_parts=[]) From f3e340fcdf6e72a992f91b16174093960052f8fc Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 1 Mar 2021 20:36:30 +0300 Subject: [PATCH 023/266] Fixing the tests --- .../integration/test_attach_without_fetching/test.py | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 2f52d3e9846..5224fa1f081 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -40,14 +40,10 @@ def check_data(nodes, detached_parts): for node in nodes: node.query("SYSTEM SYNC REPLICA test") - print("Checking data integrity for", node.name, "; parts missing:", detached_parts) - for i in range(10): assert node.query("SELECT count() FROM test WHERE n % 10 == " + str(i)) == \ "0\n" if i in detached_parts else "10\n" - print(node.query("SELECT partition_id, name FROM system.parts WHERE table='test'")) - assert node.query("SELECT count() FROM system.parts WHERE table='test'") == \ str(10 - len(detached_parts)) + "\n" @@ -73,12 +69,6 @@ def test_attach_without_fetching(start_cluster): check_data([node_1, node_2], detached_parts=[]) - path_to_detached_part_1 = node_1.query("SELECT path from system.parts WHERE table='test' AND name='1_0_0_0'") \ - .strip() - path_to_detached_part_1 = path_to_detached_part_1.split('/')[:-1] - path_to_detached_part_1 = path_to_detached_part_1[:-1] + ["detached", path_to_detached_part_1[-1]] - path_to_detached_part_1 = "/".join(path_to_detached_part_1) - # 1. Detach the first two partitions/parts on the replicas # This part will be fetched from other replicas as it would be missing in the detached/ folder and # also attached locally. @@ -101,6 +91,6 @@ def test_attach_without_fetching(start_cluster): # 4. Break the part data on the second node to corrupt the checksums. # Replica 3 should download the data from replica 1 as there is no local data. # Replica 2 should also download the data from 1 as the checksums won't match. - corrupt_part_data_by_path(node_2, path_to_detached_part_1) + corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") node_1.query("ALTER TABLE test ATTACH PARTITION 1") check_data([node_1, node_2, node_3], detached_parts=[]) From 6191580fe18e5abdf3b8cb9dd38f961f4b1904b3 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 1 Mar 2021 20:54:02 +0300 Subject: [PATCH 024/266] Added the PartitionManager to check that replica 1 attaches the local data --- tests/integration/test_attach_without_fetching/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 5224fa1f081..f72f0dc71bc 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -85,7 +85,12 @@ def test_attach_without_fetching(start_cluster): # 3. Attach the first part and check if it has been fetched correctly. # Replica 3 should download the data from replica 2 as there is no local data. # Replica 1 should attach the local data from detached/. - node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") + with PartitionManager() as pm: + # The non-initiator replica downloads the data from initiator only. + # If something goes wrong and replica 1 wants to download data from replica 2, the test will fail. + pm.partition_instances(node_1, node_2) + node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") + check_data([node_1, node_2, node_3], detached_parts=[1]) # 4. Break the part data on the second node to corrupt the checksums. From 6ea574525c8ca72db5621eb675fa2b247a986881 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 3 Mar 2021 16:51:41 +0300 Subject: [PATCH 025/266] Small fixes regarding the review --- .../statements/alter/partition.md | 18 +++++++++++------- docs/en/sql-reference/statements/system.md | 4 +++- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- .../test_attach_without_fetching/test.py | 9 +++++---- 6 files changed, 24 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 9efaafd4d54..dd24231ca3f 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -61,7 +61,8 @@ ALTER TABLE mt DROP PARTITION '2020-11-21'; ALTER TABLE mt DROP PART 'all_4_4_0'; ``` -Note: the command does NOT throw an exception on an invalid `partition_expr`. +Note: the command does NOT throw an exception if the specified part does not exist, +e.g. `ALTER TABLE mt DROP PART 'i_do_not_exist'` will succeed. ## DROP DETACHED PARTITION\|PART {#alter_drop-detached} @@ -72,7 +73,8 @@ ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr Removes the specified part or all parts of the specified partition from `detached`. Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). -Note: the command does NOT throw an exception on an invalid `partition_expr`. +Note: the command does NOT throw an exception if the specified part does not exist, +e.g. `ALTER TABLE mt DROP DETACHED PART[ITION] 'i_do_not_exist'` will succeed. ## ATTACH PARTITION\|PART {#alter_attach-partition} @@ -89,13 +91,15 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0; Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). -This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. +This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. +If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. -If the non-initiator replica, receiving the attach command, finds the part with the correct checksums in its own `detached` folder, -it attaches the data without fetching it from other replicas. -If there is no part with the correct checksums, the data is downloaded from the replica-initiator. +If the non-initiator replica, receiving the attach command, finds the part with the correct checksums in its own +`detached` folder, it attaches the data without fetching it from other replicas. +If there is no part with the correct checksums, the data is downloaded from any replica having the part. -So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas. +You can put data to the `detached` directory on one replica and use the `ALTER ... ATTACH` query to add it to the +table on all replicas. ## ATTACH PARTITION FROM {#alter_attach-partition-from} diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 22e018d6ec9..33cfffc0b12 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -264,7 +264,9 @@ Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name ``` -Note that after running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from other replicas. +After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from +the common replicated log into its own replication queue, and then the query waits till the replica processes all +of the fetched commands. ### RESTART REPLICA {#query_language-system-restart-replica} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 04dcf0be074..c1e425d3a06 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -54,7 +54,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const case ATTACH_PART: out << "attach\n" << new_part_name << "\n" - << "part_checksum: " << part_checksum << "\n"; + << "part_checksum: " << part_checksum; break; case MERGE_PARTS: @@ -189,7 +189,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) else if (type_str == "attach") { type = ATTACH_PART; - in >> new_part_name >> "\npart_checksum: " >> part_checksum >> "\n"; + in >> new_part_name >> "\npart_checksum: " >> part_checksum; } else if (type_str == "merge") { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 040c95c9639..7fd8f16816b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1339,7 +1339,8 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo auto single_disk_volume = std::make_shared("volume_" + part_name, getDiskForPart(part_name, "detached/")); - MergeTreeData::MutableDataPartPtr iter_part_ptr = createPart(part_name, single_disk_volume, part_to_path); + MergeTreeData::MutableDataPartPtr iter_part_ptr = + createPart(part_name, part_iter, single_disk_volume, part_to_path); if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) continue; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 9b5bf5a4b48..e62d90f5cee 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -84,7 +84,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const MutableColumnPtr database_column_mut = ColumnString::create(); for (const auto & database : databases) { - /// Checck if database can contain MergeTree tables, + /// Check if database can contain MergeTree tables, /// if not it's unnecessary to load all tables of database just to filter all of them. if (database.second->canContainMergeTreeTables()) database_column_mut->insert(database.first); diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index f72f0dc71bc..a9acad5a03b 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -83,15 +83,16 @@ def test_attach_without_fetching(start_cluster): fill_node(node_3) # 3. Attach the first part and check if it has been fetched correctly. - # Replica 3 should download the data from replica 2 as there is no local data. # Replica 1 should attach the local data from detached/. + # Replica 3 should download the data from replica 1 as there is no local data and other connections are broken. with PartitionManager() as pm: - # The non-initiator replica downloads the data from initiator only. - # If something goes wrong and replica 1 wants to download data from replica 2, the test will fail. + # If something goes wrong and replica 1 wants to fetch data, the test will fail. pm.partition_instances(node_1, node_2) + pm.partition_instances(node_3, node_2) + node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") - check_data([node_1, node_2, node_3], detached_parts=[1]) + check_data([node_1, node_2, node_3], detached_parts=[1]) # 4. Break the part data on the second node to corrupt the checksums. # Replica 3 should download the data from replica 1 as there is no local data. From 5cfeedb1e0b9f52ca6df0c18faf95453e4bc97fd Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 4 Mar 2021 15:21:23 +0300 Subject: [PATCH 026/266] Add setting http_max_uri_size --- src/Core/Settings.h | 1 + src/Server/HTTP/HTTPServerRequest.cpp | 6 +++--- src/Server/HTTP/HTTPServerRequest.h | 3 ++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8afc08da21a..7ec7a87dfbc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -227,6 +227,7 @@ class IColumn; M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ + M(UInt64, http_max_uri_size, 16384, "HTTP max URI length", 0) \ M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index bdba6a51d91..3f55025a139 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -15,8 +15,8 @@ namespace DB { - HTTPServerRequest::HTTPServerRequest(const Context & context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session) + : max_uri_size(context.getSettingsRef().http_max_uri_size) { response.attachRequest(this); @@ -93,10 +93,10 @@ void HTTPServerRequest::readRequest(ReadBuffer & in) skipWhitespaceIfAny(in); - while (in.read(ch) && !Poco::Ascii::isSpace(ch) && uri.size() <= MAX_URI_LENGTH) + while (in.read(ch) && !Poco::Ascii::isSpace(ch) && uri.size() <= max_uri_size) uri += ch; - if (uri.size() > MAX_URI_LENGTH) + if (uri.size() > max_uri_size) throw Poco::Net::MessageException("HTTP request URI invalid or too long"); skipWhitespaceIfAny(in); diff --git a/src/Server/HTTP/HTTPServerRequest.h b/src/Server/HTTP/HTTPServerRequest.h index 7fd54850212..a0f022f32ec 100644 --- a/src/Server/HTTP/HTTPServerRequest.h +++ b/src/Server/HTTP/HTTPServerRequest.h @@ -43,11 +43,12 @@ private: MAX_NAME_LENGTH = 256, MAX_VALUE_LENGTH = 8192, MAX_METHOD_LENGTH = 32, - MAX_URI_LENGTH = 16384, MAX_VERSION_LENGTH = 8, MAX_FIELDS_NUMBER = 100, }; + const size_t max_uri_size; + std::unique_ptr stream; Poco::Net::SocketImpl * socket; Poco::Net::SocketAddress client_address; From a3509065b21d9baa6f5b16570874b6323074e851 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 4 Mar 2021 15:43:00 +0300 Subject: [PATCH 027/266] Add trivial README about Poco-based code --- src/Server/HTTP/README.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 src/Server/HTTP/README.md diff --git a/src/Server/HTTP/README.md b/src/Server/HTTP/README.md new file mode 100644 index 00000000000..71730962780 --- /dev/null +++ b/src/Server/HTTP/README.md @@ -0,0 +1,3 @@ +# Notice + +The source code located in this folder is based on some files from the POCO project, from here `contrib/poco/Net/src`. From 91ff08452ef1a0fbacbebc208e6c9a22a2ad2e7c Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 4 Mar 2021 16:58:18 +0300 Subject: [PATCH 028/266] Remove unused code --- src/Server/HTTP/HTMLForm.cpp | 57 ------------------------ src/Server/HTTP/HTMLForm.h | 49 -------------------- src/Server/HTTP/HTTPServerConnection.cpp | 27 ----------- src/Server/HTTP/HTTPServerConnection.h | 1 - src/Server/HTTP/HTTPServerResponse.cpp | 40 ----------------- src/Server/HTTP/HTTPServerResponse.h | 21 --------- 6 files changed, 195 deletions(-) diff --git a/src/Server/HTTP/HTMLForm.cpp b/src/Server/HTTP/HTMLForm.cpp index ca407858c33..e4d13aa2218 100644 --- a/src/Server/HTTP/HTMLForm.cpp +++ b/src/Server/HTTP/HTMLForm.cpp @@ -71,23 +71,6 @@ HTMLForm::HTMLForm(const Poco::URI & uri) : field_limit(DFL_FIELD_LIMIT), value_ } -void HTMLForm::setEncoding(const std::string & encoding_) -{ - encoding = encoding_; -} - - -void HTMLForm::addPart(const std::string & name, Poco::Net::PartSource * source) -{ - poco_check_ptr(source); - - Part part; - part.name = name; - part.source = std::unique_ptr(source); - parts.push_back(std::move(part)); -} - - void HTMLForm::load(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler) { clear(); @@ -126,36 +109,12 @@ void HTMLForm::load(const Poco::Net::HTTPRequest & request, ReadBuffer & request } -void HTMLForm::load(const Poco::Net::HTTPRequest & request) -{ - NullPartHandler nah; - EmptyReadBuffer nis; - load(request, nis, nah); -} - - -void HTMLForm::read(ReadBuffer & in, PartHandler & handler) -{ - if (encoding == ENCODING_URL) - readQuery(in); - else - readMultipart(in, handler); -} - - void HTMLForm::read(ReadBuffer & in) { readQuery(in); } -void HTMLForm::read(const std::string & queryString) -{ - ReadBufferFromString istr(queryString); - readQuery(istr); -} - - void HTMLForm::readQuery(ReadBuffer & in) { size_t fields = 0; @@ -269,22 +228,6 @@ void HTMLForm::readMultipart(ReadBuffer & in_, PartHandler & handler) } -void HTMLForm::setFieldLimit(int limit) -{ - poco_assert(limit >= 0); - - field_limit = limit; -} - - -void HTMLForm::setValueLengthLimit(int limit) -{ - poco_assert(limit >= 0); - - value_length_limit = limit; -} - - HTMLForm::MultipartReadBuffer::MultipartReadBuffer(ReadBuffer & in_, const std::string & boundary_) : ReadBuffer(nullptr, 0), in(in_), boundary("--" + boundary_) { diff --git a/src/Server/HTTP/HTMLForm.h b/src/Server/HTTP/HTMLForm.h index 27be712e1d5..8d8fb0d1719 100644 --- a/src/Server/HTTP/HTMLForm.h +++ b/src/Server/HTTP/HTMLForm.h @@ -52,24 +52,6 @@ public: return (it != end()) ? DB::parse(it->second) : default_value; } - template - T getParsed(const std::string & key) - { - return DB::parse(get(key)); - } - - /// Sets the encoding used for posting the form. - /// Encoding must be either "application/x-www-form-urlencoded" (which is the default) or "multipart/form-data". - void setEncoding(const std::string & encoding); - - /// Returns the encoding used for posting the form. - const std::string & getEncoding() const { return encoding; } - - /// Adds an part/attachment (file upload) to the form. - /// The form takes ownership of the PartSource and deletes it when it is no longer needed. - /// The part will only be sent if the encoding set for the form is "multipart/form-data" - void addPart(const std::string & name, Poco::Net::PartSource * pSource); - /// Reads the form data from the given HTTP request. /// Uploaded files are passed to the given PartHandler. void load(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler); @@ -78,41 +60,10 @@ public: /// Uploaded files are silently discarded. void load(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody); - /// Reads the form data from the given HTTP request. - /// The request must be a GET request and the form data must be in the query string (URL encoded). - /// For POST requests, you must use one of the overloads taking an additional input stream for the request body. - void load(const Poco::Net::HTTPRequest & request); - - /// Reads the form data from the given input stream. - /// The form data read from the stream must be in the encoding specified for the form. - /// Note that read() does not clear the form before reading the new values. - void read(ReadBuffer & in, PartHandler & handler); - /// Reads the URL-encoded form data from the given input stream. /// Note that read() does not clear the form before reading the new values. void read(ReadBuffer & in); - /// Reads the form data from the given HTTP query string. - /// Note that read() does not clear the form before reading the new values. - void read(const std::string & queryString); - - /// Returns the MIME boundary used for writing multipart form data. - const std::string & getBoundary() const { return boundary; } - - /// Returns the maximum number of header fields allowed. - /// See setFieldLimit() for more information. - int getFieldLimit() const { return field_limit; } - - /// Sets the maximum number of header fields allowed. This limit is used to defend certain kinds of denial-of-service attacks. - /// Specify 0 for unlimited (not recommended). The default limit is 100. - void setFieldLimit(int limit); - - /// Sets the maximum size for form field values stored as strings. - void setValueLengthLimit(int limit); - - /// Returns the maximum size for form field values stored as strings. - int getValueLengthLimit() const { return value_length_limit; } - static const std::string ENCODING_URL; /// "application/x-www-form-urlencoded" static const std::string ENCODING_MULTIPART; /// "multipart/form-data" static const int UNKNOWN_CONTENT_LENGTH; diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index e2ee4c8882b..be2ca4b87d9 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -98,31 +98,4 @@ void HTTPServerConnection::sendErrorResponse(Poco::Net::HTTPServerSession & sess session.setKeepAlive(false); } -void HTTPServerConnection::onServerStopped(const bool & abortCurrent) -{ - stopped = true; - if (abortCurrent) - { - try - { - socket().shutdown(); - } - catch (...) - { - } - } - else - { - std::unique_lock lock(mutex); - - try - { - socket().shutdown(); - } - catch (...) - { - } - } -} - } diff --git a/src/Server/HTTP/HTTPServerConnection.h b/src/Server/HTTP/HTTPServerConnection.h index 589c33025bf..55b6e921d9f 100644 --- a/src/Server/HTTP/HTTPServerConnection.h +++ b/src/Server/HTTP/HTTPServerConnection.h @@ -23,7 +23,6 @@ public: protected: static void sendErrorResponse(Poco::Net::HTTPServerSession & session, Poco::Net::HTTPResponse::HTTPStatus status); - void onServerStopped(const bool & abortCurrent); private: Context context; diff --git a/src/Server/HTTP/HTTPServerResponse.cpp b/src/Server/HTTP/HTTPServerResponse.cpp index e3d52fffa80..db5cfb132e3 100644 --- a/src/Server/HTTP/HTTPServerResponse.cpp +++ b/src/Server/HTTP/HTTPServerResponse.cpp @@ -94,32 +94,6 @@ std::pair, std::shared_ptr> HTTPServ return std::make_pair(header_stream, stream); } -void HTTPServerResponse::sendFile(const std::string & path, const std::string & mediaType) -{ - poco_assert(!stream); - - Poco::File f(path); - Poco::Timestamp date_time = f.getLastModified(); - Poco::File::FileSize length = f.getSize(); - set("Last-Modified", Poco::DateTimeFormatter::format(date_time, Poco::DateTimeFormat::HTTP_FORMAT)); - setContentLength64(length); - setContentType(mediaType); - setChunkedTransferEncoding(false); - - Poco::FileInputStream istr(path); - if (istr.good()) - { - stream = std::make_shared(session); - write(*stream); - if (request && request->getMethod() != HTTPRequest::HTTP_HEAD) - { - Poco::StreamCopier::copyStream(istr, *stream); - } - } - else - throw Poco::OpenFileException(path); -} - void HTTPServerResponse::sendBuffer(const void * buffer, std::size_t length) { poco_assert(!stream); @@ -135,20 +109,6 @@ void HTTPServerResponse::sendBuffer(const void * buffer, std::size_t length) } } -void HTTPServerResponse::redirect(const std::string & uri, HTTPStatus status) -{ - poco_assert(!stream); - - setContentLength(0); - setChunkedTransferEncoding(false); - - setStatusAndReason(status); - set("Location", uri); - - stream = std::make_shared(session); - write(*stream); -} - void HTTPServerResponse::requireAuthentication(const std::string & realm) { poco_assert(!stream); diff --git a/src/Server/HTTP/HTTPServerResponse.h b/src/Server/HTTP/HTTPServerResponse.h index 82221ce3a83..32c74d40b13 100644 --- a/src/Server/HTTP/HTTPServerResponse.h +++ b/src/Server/HTTP/HTTPServerResponse.h @@ -36,17 +36,6 @@ public: /// or redirect() has been called. std::pair, std::shared_ptr> beginSend(); /// TODO: use some WriteBuffer implementation here. - /// Sends the response header to the client, followed - /// by the content of the given file. - /// - /// Must not be called after send(), sendBuffer() - /// or redirect() has been called. - /// - /// Throws a FileNotFoundException if the file - /// cannot be found, or an OpenFileException if - /// the file cannot be opened. - void sendFile(const std::string & path, const std::string & mediaType); - /// Sends the response header to the client, followed /// by the contents of the given buffer. /// @@ -61,16 +50,6 @@ public: /// or redirect() has been called. void sendBuffer(const void * pBuffer, std::size_t length); /// FIXME: do we need this one? - /// Sets the status code, which must be one of - /// HTTP_MOVED_PERMANENTLY (301), HTTP_FOUND (302), - /// or HTTP_SEE_OTHER (303), - /// and sets the "Location" header field - /// to the given URI, which according to - /// the HTTP specification, must be absolute. - /// - /// Must not be called after send() has been called. - void redirect(const std::string & uri, Poco::Net::HTTPResponse::HTTPStatus status = Poco::Net::HTTPResponse::HTTP_FOUND); - void requireAuthentication(const std::string & realm); /// Sets the status code to 401 (Unauthorized) /// and sets the "WWW-Authenticate" header field From 6a07ee15066f1da22332e2309246dc6dacffb4be Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 4 Mar 2021 17:01:37 +0300 Subject: [PATCH 029/266] CC --- src/Server/HTTP/HTTPServer.h | 6 +++--- src/Server/HTTP/HTTPServerConnection.cpp | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Server/HTTP/HTTPServer.h b/src/Server/HTTP/HTTPServer.h index 1ce62c65ca2..3d2a2ac9cf4 100644 --- a/src/Server/HTTP/HTTPServer.h +++ b/src/Server/HTTP/HTTPServer.h @@ -19,7 +19,7 @@ public: explicit HTTPServer( const Context & context, HTTPRequestHandlerFactoryPtr factory, - UInt16 portNumber = 80, + UInt16 port_number = 80, Poco::Net::HTTPServerParams::Ptr params = new Poco::Net::HTTPServerParams); HTTPServer( @@ -31,13 +31,13 @@ public: HTTPServer( const Context & context, HTTPRequestHandlerFactoryPtr factory, - Poco::ThreadPool & threadPool, + Poco::ThreadPool & thread_pool, const Poco::Net::ServerSocket & socket, Poco::Net::HTTPServerParams::Ptr params); ~HTTPServer() override; - void stopAll(bool abortCurrent = false); + void stopAll(bool abort_current = false); private: HTTPRequestHandlerFactoryPtr factory; diff --git a/src/Server/HTTP/HTTPServerConnection.cpp b/src/Server/HTTP/HTTPServerConnection.cpp index be2ca4b87d9..7a6cd4cab54 100644 --- a/src/Server/HTTP/HTTPServerConnection.cpp +++ b/src/Server/HTTP/HTTPServerConnection.cpp @@ -67,15 +67,15 @@ void HTTPServerConnection::run() } } } - catch (Poco::Net::NoMessageException &) + catch (const Poco::Net::NoMessageException &) { break; } - catch (Poco::Net::MessageException &) + catch (const Poco::Net::MessageException &) { sendErrorResponse(session, Poco::Net::HTTPResponse::HTTP_BAD_REQUEST); } - catch (Poco::Exception &) + catch (const Poco::Exception &) { if (session.networkException()) { From a4bd6939c94ec5daf6b437b3dd3c8bfa6a109263 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 9 Mar 2021 23:46:47 +0300 Subject: [PATCH 030/266] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/postgresql.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- docs/en/sql-reference/table-functions/index.md | 2 +- docs/en/sql-reference/table-functions/postgresql.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 48e183b6d32..7b8414246dc 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -74,4 +74,4 @@ SELECT * FROM test_database.test_table; - [The 'postgresql' table function](../../../sql-reference/table-functions/postgresql.md) - [Using PostgreSQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) -[Original article](https://clickhouse.tech/docs/en/operations/table_engines/postgresql/) +[Original article](https://clickhouse.tech/docs/en/operations/table-engines/integrations/postgresql/) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 65ad141979d..169a35d539d 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -723,4 +723,4 @@ Setting fields: - `table` — Remote table name. - `invalidate_query` — Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). -[Original article](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_sources/) +[Original article](https://clickhouse.tech/docs/en/sql-reference/dictionaries/external-dictionaries/external-dict-sources) diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index 38d98f11402..75e88fdf50d 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -35,4 +35,4 @@ You can use table functions in: | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | | [postgresql](../../sql-reference/table-functions/postgresql.md) | Creates a [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)-engine table. | -[Original article](https://clickhouse.tech/docs/en/query_language/table_functions/) +[Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/) diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 60d813a08e7..99f209ab08e 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -70,4 +70,4 @@ SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postg - [The PostgreSQL table engine](../../engines/table-engines/integrations/postgresql.md) - [Using PostgreSQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) -[Original article](https://clickhouse.tech/docs/en/sql-reference/table_functions/postgresql/) +[Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/postgresql/) From 8e00ff78987f41f5fcd767af067cdbbea6517355 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 13 Mar 2021 04:25:13 +0300 Subject: [PATCH 031/266] try run two nodes with Replicated db and NuKeeper --- docker/test/stateless/run.sh | 49 ++++++++++++- tests/clickhouse-test | 7 +- tests/config/config.d/database_replicated.xml | 69 +++++++++++++++++++ tests/config/install.sh | 3 + 4 files changed, 124 insertions(+), 4 deletions(-) create mode 100644 tests/config/config.d/database_replicated.xml diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3119ae27c59..0d67df8226c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,6 +11,20 @@ dpkg -i package_folder/clickhouse-test_*.deb # install test configs /usr/share/clickhouse-test/config/install.sh +#cp /use_test_keeper.xml /etc/clickhouse-server/config.d/zookeeper.xml +#cp /enable_test_keeper1.xml /etc/clickhouse-server/config.d/test_keeper_port.xml +#cp /clusters2.xml /etc/clickhouse-server/config.d/ + +#mkdir /etc/clickhouse-server2 +#chown clickhouse /etc/clickhouse-server2 +#chgrp clickhouse /etc/clickhouse-server2 +#sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 +#rm /etc/clickhouse-server2/config.d/macros.xml +#sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + +#cat /usr/bin/clickhouse-test | sed "s| ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')| ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')|" > /usr/bin/clickhouse-test-tmp +#mv /usr/bin/clickhouse-test-tmp /usr/bin/clickhouse-test +#chmod a+x /usr/bin/clickhouse-test # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then @@ -34,11 +48,33 @@ if [ "$NUM_TRIES" -gt "1" ]; then # simpliest way to forward env variables to server sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon - sleep 5 else - service clickhouse-server start && sleep 5 + service clickhouse-server start fi +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 + # And we have to copy configs... + mkdir /etc/clickhouse-server2 + chown clickhouse /etc/clickhouse-server2 + chgrp clickhouse /etc/clickhouse-server2 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 + rm /etc/clickhouse-server2/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + + sudo mkdir /var/lib/clickhouse2 + sudo chmod a=rwx /var/lib/clickhouse2 + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --mysql_port 19004 \ + --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ + --macros.replica r2 # It doesn't work :( +fi + +sleep 5 + if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi @@ -68,6 +104,10 @@ function run_tests() | tee -a test_output/test_result.txt } +#clickhouse-client --port 9000 -q "SELECT * FROM system.macros" +#clickhouse-client --port 19000 -q "SELECT * FROM system.macros" +#clickhouse-client --port 19000 -q "SELECT 2" + export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: @@ -81,3 +121,8 @@ if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then fi tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: + +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: +fi diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 198c8bbe0c9..212503a3e71 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,7 +107,7 @@ def remove_control_characters(s): def get_db_engine(args, database_name): if args.replicated_database: - return " ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) + return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine @@ -172,7 +172,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10) try: - clickhouse_proc_create.communicate(("DROP DATABASE " + database), timeout=seconds_left) + drop_database_query = "DROP DATABASE " + database + if args.replicated_database: + drop_database_query += " ON CLUSTER test_cluster_database_replicated" + clickhouse_proc_create.communicate((drop_database_query), timeout=seconds_left) except TimeoutExpired: # kill test process because it can also hung if proc.returncode is None: diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml new file mode 100644 index 00000000000..7392968b415 --- /dev/null +++ b/tests/config/config.d/database_replicated.xml @@ -0,0 +1,69 @@ + + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + + + + 9181 + 1 + + + 5000 + 10000 + trace + + + + + 1 + localhost + 44444 + true + 3 + + + 2 + localhost + 44445 + true + true + 2 + + + 3 + localhost + 44446 + true + true + 1 + + + + + + + + + localhost + 9000 + + + localhost + 19000 + + + + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 1fca2b11e04..d0f978a3315 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -65,6 +65,9 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/ + ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ + rm /etc/clickhouse-server/config.d/zookeeper.xml + rm /etc/clickhouse-server/config.d/test_keeper_port.xml fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From e62e256e77a99d6ceaf4b2b2efb98e491c634a0f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 13 Mar 2021 13:22:48 +0300 Subject: [PATCH 032/266] fix --- docker/test/stateful/run.sh | 22 +++++++++++ docker/test/stateless/clusters2.xml | 22 +++++++++++ docker/test/stateless/enable_test_keeper1.xml | 38 +++++++++++++++++++ docker/test/stateless/run.sh | 38 +++++-------------- docker/test/stateless/use_test_keeper.xml | 16 ++++++++ tests/clickhouse-test | 2 +- tests/config/config.d/database_replicated.xml | 6 +++ tests/config/install.sh | 22 +++++++++++ 8 files changed, 137 insertions(+), 29 deletions(-) create mode 100644 docker/test/stateless/clusters2.xml create mode 100644 docker/test/stateless/enable_test_keeper1.xml create mode 100644 docker/test/stateless/use_test_keeper.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 6b90a9e7e37..195cfbfa3d3 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -13,6 +13,22 @@ dpkg -i package_folder/clickhouse-test_*.deb function start() { + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --mysql_port 19004 \ + --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 + + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ + --mysql_port 29004 \ + --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 + fi + counter=0 until clickhouse-client --query "SELECT 1" do @@ -73,3 +89,9 @@ mv /var/log/clickhouse-server/stderr.log /test_output/ ||: if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: fi +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: + mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: +fi diff --git a/docker/test/stateless/clusters2.xml b/docker/test/stateless/clusters2.xml new file mode 100644 index 00000000000..73b6274df12 --- /dev/null +++ b/docker/test/stateless/clusters2.xml @@ -0,0 +1,22 @@ + + + + + + localhost + 9000 + + + localhost + 19000 + + + + + localhost + 29000 + + + + + diff --git a/docker/test/stateless/enable_test_keeper1.xml b/docker/test/stateless/enable_test_keeper1.xml new file mode 100644 index 00000000000..6c9669a42fd --- /dev/null +++ b/docker/test/stateless/enable_test_keeper1.xml @@ -0,0 +1,38 @@ + + + 9181 + 1 + + + 5000 + 10000 + trace + + + + + 1 + localhost + 44444 + true + 3 + + + 2 + localhost + 44445 + true + true + 2 + + + 3 + localhost + 44446 + true + true + 1 + + + + diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0d67df8226c..dfa05e1a354 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,20 +11,6 @@ dpkg -i package_folder/clickhouse-test_*.deb # install test configs /usr/share/clickhouse-test/config/install.sh -#cp /use_test_keeper.xml /etc/clickhouse-server/config.d/zookeeper.xml -#cp /enable_test_keeper1.xml /etc/clickhouse-server/config.d/test_keeper_port.xml -#cp /clusters2.xml /etc/clickhouse-server/config.d/ - -#mkdir /etc/clickhouse-server2 -#chown clickhouse /etc/clickhouse-server2 -#chgrp clickhouse /etc/clickhouse-server2 -#sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 -#rm /etc/clickhouse-server2/config.d/macros.xml -#sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - -#cat /usr/bin/clickhouse-test | sed "s| ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')| ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')|" > /usr/bin/clickhouse-test-tmp -#mv /usr/bin/clickhouse-test-tmp /usr/bin/clickhouse-test -#chmod a+x /usr/bin/clickhouse-test # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then @@ -53,17 +39,7 @@ else fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 - # And we have to copy configs... - mkdir /etc/clickhouse-server2 - chown clickhouse /etc/clickhouse-server2 - chgrp clickhouse /etc/clickhouse-server2 - sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 - rm /etc/clickhouse-server2/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - sudo mkdir /var/lib/clickhouse2 - sudo chmod a=rwx /var/lib/clickhouse2 sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ @@ -71,6 +47,14 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( + + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ + --mysql_port 29004 \ + --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ + --macros.shard s2 # It doesn't work :( fi sleep 5 @@ -104,10 +88,6 @@ function run_tests() | tee -a test_output/test_result.txt } -#clickhouse-client --port 9000 -q "SELECT * FROM system.macros" -#clickhouse-client --port 19000 -q "SELECT * FROM system.macros" -#clickhouse-client --port 19000 -q "SELECT 2" - export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: @@ -124,5 +104,7 @@ tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_l if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: + mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/docker/test/stateless/use_test_keeper.xml b/docker/test/stateless/use_test_keeper.xml new file mode 100644 index 00000000000..26ec47e50e8 --- /dev/null +++ b/docker/test/stateless/use_test_keeper.xml @@ -0,0 +1,16 @@ + + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + + diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 212503a3e71..6d7ca807992 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,7 +107,7 @@ def remove_control_characters(s): def get_db_engine(args, database_name): if args.replicated_database: - return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) + return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')".format(database_name) if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 7392968b415..5b09a13ca4b 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -63,6 +63,12 @@ 19000 + + + localhost + 29000 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index d0f978a3315..76cca9956cb 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -68,6 +68,28 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ rm /etc/clickhouse-server/config.d/zookeeper.xml rm /etc/clickhouse-server/config.d/test_keeper_port.xml + + # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 + # And we have to copy configs... + mkdir /etc/clickhouse-server2 + mkdir /etc/clickhouse-server3 + chown clickhouse /etc/clickhouse-server2 + chown clickhouse /etc/clickhouse-server3 + chgrp clickhouse /etc/clickhouse-server2 + chgrp clickhouse /etc/clickhouse-server3 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server3 + rm /etc/clickhouse-server2/config.d/macros.xml + rm /etc/clickhouse-server3/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server3/config.d/macros.xml + + sudo mkdir /var/lib/clickhouse2 + sudo mkdir /var/lib/clickhouse3 + sudo chown clickhouse /var/lib/clickhouse2 + sudo chown clickhouse /var/lib/clickhouse3 + sudo chgrp clickhouse /var/lib/clickhouse2 + sudo chgrp clickhouse /var/lib/clickhouse3 fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From 7f2561b853f8ad302e0bba5d54e5cc1009ac2848 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 14 Mar 2021 20:11:26 +0300 Subject: [PATCH 033/266] In russian --- .../en/engines/database-engines/postgresql.md | 4 +- docs/ru/engines/database-engines/index.md | 6 +- .../ru/engines/database-engines/postgresql.md | 138 ++++++++++++++++++ docs/ru/engines/table-engines/index.md | 21 ++- .../integrations/embedded-rocksdb.md | 2 +- .../table-engines/integrations/hdfs.md | 2 +- .../table-engines/integrations/jdbc.md | 2 +- .../table-engines/integrations/kafka.md | 2 +- .../table-engines/integrations/mongodb.md | 2 +- .../table-engines/integrations/mysql.md | 2 +- .../table-engines/integrations/odbc.md | 2 +- docs/ru/engines/table-engines/postgresql.md | 77 ++++++++++ .../external-dicts-dict-sources.md | 51 ++++++- .../ru/sql-reference/table-functions/index.md | 25 ++-- .../table-functions/postgresql.md | 73 +++++++++ 15 files changed, 378 insertions(+), 31 deletions(-) create mode 100644 docs/ru/engines/database-engines/postgresql.md create mode 100644 docs/ru/engines/table-engines/postgresql.md create mode 100644 docs/ru/sql-reference/table-functions/postgresql.md diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md index d21eb68bbdb..1fa86b7ac21 100644 --- a/docs/en/engines/database-engines/postgresql.md +++ b/docs/en/engines/database-engines/postgresql.md @@ -5,7 +5,7 @@ toc_title: PostgreSQL # PostgreSQL {#postgresql} -Allows to connect to databases on a remote PostgreSQL server. Supports read and write operations (`SELECT` and `INSERT` queries) to exchange data between ClickHouse and PostgreSQL. +Allows to connect to databases on a remote [PostgreSQL](https://www.postgresql.org) server. Supports read and write operations (`SELECT` and `INSERT` queries) to exchange data between ClickHouse and PostgreSQL. Gives the real-time access to table list and table structure from remote PostgreSQL with the help of `SHOW TABLES` and `DESCRIBE TABLE` queries. @@ -135,4 +135,4 @@ DESCRIBE TABLE test_database.test_table; └────────┴───────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/database_engines/postgresql/) +[Original article](https://clickhouse.tech/docs/en/database-engines/postgresql/) diff --git a/docs/ru/engines/database-engines/index.md b/docs/ru/engines/database-engines/index.md index 4dfe766f066..eee222d5f07 100644 --- a/docs/ru/engines/database-engines/index.md +++ b/docs/ru/engines/database-engines/index.md @@ -4,7 +4,7 @@ toc_priority: 27 toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" --- -# Движки баз данных {#dvizhki-baz-dannykh} +# Движки баз данных {#database-engines} Движки баз данных обеспечивают работу с таблицами. @@ -18,4 +18,8 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" - [Lazy](../../engines/database-engines/lazy.md) +- [Atomic](../../engines/database-engines/atomic.md) + +- [PostgreSQL](../../engines/database-engines/postgresql.md) + [Оригинальная статья](https://clickhouse.tech/docs/ru/database_engines/) diff --git a/docs/ru/engines/database-engines/postgresql.md b/docs/ru/engines/database-engines/postgresql.md new file mode 100644 index 00000000000..afda3feec5a --- /dev/null +++ b/docs/ru/engines/database-engines/postgresql.md @@ -0,0 +1,138 @@ +--- +toc_priority: 35 +toc_title: PostgreSQL +--- + +# PostgreSQL {#postgresql} + +Позволяет подключаться к БД на удаленном сервере [PostgreSQL](https://www.postgresql.org). Поддерживает операции чтения и записи (запросы `SELECT` и `INSERT`) для обмена данными между ClickHouse и PostgreSQL. + +Позволяет в реальном времени получать от удаленного сервера PostgreSQL информацию о таблицах БД и их структуре с помощью запросов `SHOW TABLES` и `DESCRIBE TABLE`. + +Поддерживает операции изменения структуры таблиц (`ALTER TABLE ... ADD|DROP COLUMN`). Если параметр `use_table_cache` (см. ниже раздел Параметры движка) установлен в значение `1`, структура таблицы кешируется, и изменения в структуре не отображаются мгновенно, но могут быть получены с помощью запросов `DETACH` и `ATTACH`. + +## Создание БД {#creating-a-database} + +``` sql +CREATE DATABASE test_database +ENGINE = PostgreSQL('host:port', 'database', 'user', 'password'[, `use_table_cache`]); +``` + +**Параметры движка** + +- `host:port` — адрес сервера PostgreSQL. +- `database` — имя удаленной БД. +- `user` — пользователь PostgreSQL. +- `password` — пароль пользователя. +- `use_table_cache` — определяет кеширование структуры таблиц БД. Необязательный параметр. Значение по умолчанию: `0`. + +## Поддерживаемые типы данных {#data_types-support} + +| PostgerSQL | ClickHouse | +|------------------|--------------------------------------------------------------| +| DATE | [Date](../../sql-reference/data-types/date.md) | +| TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) | +| REAL | [Float32](../../sql-reference/data-types/float.md) | +| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | +| DECIMAL, NUMERIC | [Decimal](../../sql-reference/data-types/decimal.md) | +| SMALLINT | [Int16](../../sql-reference/data-types/int-uint.md) | +| INTEGER | [Int32](../../sql-reference/data-types/int-uint.md) | +| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) | +| SERIAL | [UInt32](../../sql-reference/data-types/int-uint.md) | +| BIGSERIAL | [UInt64](../../sql-reference/data-types/int-uint.md) | +| TEXT, CHAR | [String](../../sql-reference/data-types/string.md) | +| INTEGER | Nullable([Int32](../../sql-reference/data-types/int-uint.md))| +| ARRAY | [Array](../../sql-reference/data-types/array.md) | + + +## Примеры использования {#examples-of-use} + +Обмен данными между БД ClickHouse и сервером PostgreSQL: + +``` sql +CREATE DATABASE test_database +ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword', 1); +``` + +``` sql +SHOW DATABASES; +``` + +``` text +┌─name──────────┐ +│ default │ +│ test_database │ +│ system │ +└───────────────┘ +``` + +``` sql +SHOW TABLES FROM test_database; +``` + +``` text +┌─name───────┐ +│ test_table │ +└────────────┘ +``` + +Чтение данных из таблицы PostgreSQL: + +``` sql +SELECT * FROM test_database.test_table; +``` + +``` text +┌─id─┬─value─┐ +│ 1 │ 2 │ +└────┴───────┘ +``` + +Запись данных в таблицу PostgreSQL: + +``` sql +INSERT INTO test_database.test_table VALUES (3,4); +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +Пусть структура таблицы была изменена в PostgreSQL: + +``` sql +postgre> ALTER TABLE test_table ADD COLUMN data Text +``` + +Поскольку при создании БД параметр `use_table_cache` был установлен в значение `1`, структура таблицы в ClickHouse была кеширована и поэтому не изменилась: + +``` sql +DESCRIBE TABLE test_database.test_table; +``` +``` text +┌─name───┬─type──────────────┐ +│ id │ Nullable(Integer) │ +│ value │ Nullable(Integer) │ +└────────┴───────────────────┘ +``` + +После того как таблицу «отцепили» и затем снова «прицепили», структура обновилась: + +``` sql +DETACH TABLE test_database.test_table; +ATTACH TABLE test_database.test_table; +DESCRIBE TABLE test_database.test_table; +``` +``` text +┌─name───┬─type──────────────┐ +│ id │ Nullable(Integer) │ +│ value │ Nullable(Integer) │ +│ data │ Nullable(String) │ +└────────┴───────────────────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/database-engines/postgresql/) diff --git a/docs/ru/engines/table-engines/index.md b/docs/ru/engines/table-engines/index.md index 740588c50a4..c6d1a94f14e 100644 --- a/docs/ru/engines/table-engines/index.md +++ b/docs/ru/engines/table-engines/index.md @@ -1,6 +1,6 @@ --- toc_folder_title: "\u0414\u0432\u0438\u0436\u043a\u0438\u0020\u0442\u0430\u0431\u043b\u0438\u0446" -toc_priority: 26 +toc_priority: 1 toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" --- @@ -16,7 +16,7 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" - Возможно ли многопоточное выполнение запроса. - Параметры репликации данных. -## Семейства движков {#semeistva-dvizhkov} +## Семейства движков {#engine-families} ### MergeTree {#mergetree} @@ -42,18 +42,23 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" - [StripeLog](log-family/stripelog.md#stripelog) - [Log](log-family/log.md#log) -### Движки для интеграции {#dvizhki-dlia-integratsii} +### Движки для интеграции {#integration-engines} Движки для связи с другими системами хранения и обработки данных. Движки семейства: -- [Kafka](integrations/kafka.md#kafka) -- [MySQL](integrations/mysql.md#mysql) -- [ODBC](integrations/odbc.md#table-engine-odbc) -- [JDBC](integrations/jdbc.md#table-engine-jdbc) +- [ODBC](../../engines/table-engines/integrations/odbc.md) +- [JDBC](../../engines/table-engines/integrations/jdbc.md) +- [MySQL](../../engines/table-engines/integrations/mysql.md) +- [MongoDB](../../engines/table-engines/integrations/mongodb.md) +- [HDFS](../../engines/table-engines/integrations/hdfs.md) +- [Kafka](../../engines/table-engines/integrations/kafka.md) +- [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) +- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) +- [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) -### Специальные движки {#spetsialnye-dvizhki} +### Специальные движки {#special-engines} Движки семейства: diff --git a/docs/ru/engines/table-engines/integrations/embedded-rocksdb.md b/docs/ru/engines/table-engines/integrations/embedded-rocksdb.md index 9b68bcfc770..1e6ec116efd 100644 --- a/docs/ru/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/ru/engines/table-engines/integrations/embedded-rocksdb.md @@ -1,5 +1,5 @@ --- -toc_priority: 6 +toc_priority: 9 toc_title: EmbeddedRocksDB --- diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index bd8e760fce4..6e97ee4058a 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -1,5 +1,5 @@ --- -toc_priority: 4 +toc_priority: 6 toc_title: HDFS --- diff --git a/docs/ru/engines/table-engines/integrations/jdbc.md b/docs/ru/engines/table-engines/integrations/jdbc.md index d7d438e0633..8a236ca50f3 100644 --- a/docs/ru/engines/table-engines/integrations/jdbc.md +++ b/docs/ru/engines/table-engines/integrations/jdbc.md @@ -1,5 +1,5 @@ --- -toc_priority: 2 +toc_priority: 3 toc_title: JDBC --- diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index 940fee2452b..8076b33a4c9 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -1,5 +1,5 @@ --- -toc_priority: 5 +toc_priority: 8 toc_title: Kafka --- diff --git a/docs/ru/engines/table-engines/integrations/mongodb.md b/docs/ru/engines/table-engines/integrations/mongodb.md index 0765b3909de..aa05563911c 100644 --- a/docs/ru/engines/table-engines/integrations/mongodb.md +++ b/docs/ru/engines/table-engines/integrations/mongodb.md @@ -1,5 +1,5 @@ --- -toc_priority: 7 +toc_priority: 5 toc_title: MongoDB --- diff --git a/docs/ru/engines/table-engines/integrations/mysql.md b/docs/ru/engines/table-engines/integrations/mysql.md index 3370e9b06d0..c64cc0110ea 100644 --- a/docs/ru/engines/table-engines/integrations/mysql.md +++ b/docs/ru/engines/table-engines/integrations/mysql.md @@ -1,5 +1,5 @@ --- -toc_priority: 3 +toc_priority: 4 toc_title: MySQL --- diff --git a/docs/ru/engines/table-engines/integrations/odbc.md b/docs/ru/engines/table-engines/integrations/odbc.md index 97317d647c8..599ef277aaf 100644 --- a/docs/ru/engines/table-engines/integrations/odbc.md +++ b/docs/ru/engines/table-engines/integrations/odbc.md @@ -1,5 +1,5 @@ --- -toc_priority: 1 +toc_priority: 2 toc_title: ODBC --- diff --git a/docs/ru/engines/table-engines/postgresql.md b/docs/ru/engines/table-engines/postgresql.md new file mode 100644 index 00000000000..f6a357a140d --- /dev/null +++ b/docs/ru/engines/table-engines/postgresql.md @@ -0,0 +1,77 @@ +--- +toc_priority: 11 +toc_title: PostgreSQL +--- + +#PostgreSQL {#postgresql} + +Движок PostgreSQL позволяет выполнять запросы `SELECT` и `INSERT` для таблиц на удаленном сервере PostgreSQL. + +## Создание таблицы {#creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + ... +) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); +``` + +См. подробное описание запроса [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query). + +**Параметры движка** + +- `host:port` — адрес сервера MySQL. + +- `database` — имя удаленной БД. + +- `table` — имя удаленной таблицы БД. + +- `user` — пользователь MySQL. + +- `password` — пароль пользователя. + +## Примеры использования {#usage-example} + +Рассмотрим таблицу ClickHouse, которая получает данные из таблицы PostgreSQL: + +``` sql +CREATE TABLE test_table +( + `int_id` Int32, + 'value' Int32 +) +ENGINE = PostgreSQL('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` sql +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` + +Добавление данных из таблицы ClickHouse в таблицу PosegreSQL: + +``` sql +INSERT INTO test_database.test_table VALUES (3,4); +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +## См. также {#see-also} + +- [Функция 'postgresql'](../../../sql-reference/table-functions/postgresql.md) +- [Пример подключения PostgreSQL как источника внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) + +[Оригинальная статья](https://clickhouse.tech/docs/en/operations/table-engines/integrations/postgresql/) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 3bb11b638b2..6e546adab78 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -68,6 +68,7 @@ SETTINGS(format_csv_allow_single_quotes = 0) - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) - [Redis](#dicts-external_dicts_dict_sources-redis) + - [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql) ## Локальный файл {#dicts-external_dicts_dict_sources-local_file} @@ -624,4 +625,52 @@ SOURCE(REDIS( - `storage_type` – способ хранения ключей. Необходимо использовать `simple` для источников с одним столбцом ключей, `hash_map` – для источников с двумя столбцами ключей. Источники с более, чем двумя столбцами ключей, не поддерживаются. Может отсутствовать, значение по умолчанию `simple`. - `db_index` – номер базы данных. Может отсутствовать, значение по умолчанию 0. -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/dicts/external_dicts_dict_sources/) +### PostgreSQL {#dicts-external_dicts_dict_sources-postgresql} + +Пример настроек: + +``` xml + + + clickhouse + localhost + 5432 + postgres + mysecretpassword + test_table
+ SELECT value FROM test_table WHERE id = 0 +
+ + + + + + + id + UInt32 + + + id + UInt32 + + + + value + UInt32 + + + +1 +``` + +Описание настроек: + +- `db` — имя удаленной БД. +- `host` — сервер PostgreSQL. +- `port` – порт на сервере PostgreSQL. Если порт не указан, используется порт по умолчанию. +- `user` — пользователь PostgreSQL. +- `password` — пароль пользователя. +- `table` — имя таблицы удаленной БД. +- `invalidate_query` — Запрос для проверки статуса словаря. Необязательный параметр. См. подробнееRead в разделе [Updating dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/dictionaries/external-dicts-dict-sources/) diff --git a/docs/ru/sql-reference/table-functions/index.md b/docs/ru/sql-reference/table-functions/index.md index 83225d54e60..f3dd00ac1bc 100644 --- a/docs/ru/sql-reference/table-functions/index.md +++ b/docs/ru/sql-reference/table-functions/index.md @@ -22,16 +22,17 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043D\u0438\u0435" !!! warning "Предупреждение" Если настройка [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) выключена, то использовать табличные функции невозможно. -| Функция | Описание | -|-----------------------|---------------------------------------------------------------------------------------------------------------------------------------| -| [file](file.md) | Создаёт таблицу с движком [File](../../engines/table-engines/special/file.md). | -| [merge](merge.md) | Создаёт таблицу с движком [Merge](../../engines/table-engines/special/merge.md). | -| [numbers](numbers.md) | Создаёт таблицу с единственным столбцом, заполненным целыми числами. | -| [remote](remote.md) | Предоставляет доступ к удалённым серверам, не создавая таблицу с движком [Distributed](../../engines/table-engines/special/distributed.md). | -| [url](url.md) | Создаёт таблицу с движком [Url](../../engines/table-engines/special/url.md). | -| [mysql](mysql.md) | Создаёт таблицу с движком [MySQL](../../engines/table-engines/integrations/mysql.md). | -| [jdbc](jdbc.md) | Создаёт таблицу с дижком [JDBC](../../engines/table-engines/integrations/jdbc.md). | -| [odbc](odbc.md) | Создаёт таблицу с движком [ODBC](../../engines/table-engines/integrations/odbc.md). | -| [hdfs](hdfs.md) | Создаёт таблицу с движком [HDFS](../../engines/table-engines/integrations/hdfs.md). | +| Функция | Описание | +|------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------| +| [file](file.md) | Создаёт таблицу с движком [File](../../engines/table-engines/special/file.md). | +| [merge](merge.md) | Создаёт таблицу с движком [Merge](../../engines/table-engines/special/merge.md). | +| [numbers](numbers.md) | Создаёт таблицу с единственным столбцом, заполненным целыми числами. | +| [remote](remote.md) | Предоставляет доступ к удалённым серверам, не создавая таблицу с движком [Distributed](../../engines/table-engines/special/distributed.md). | +| [url](url.md) | Создаёт таблицу с движком [Url](../../engines/table-engines/special/url.md). | +| [mysql](mysql.md) | Создаёт таблицу с движком [MySQL](../../engines/table-engines/integrations/mysql.md). | +| [jdbc](jdbc.md) | Создаёт таблицу с дижком [JDBC](../../engines/table-engines/integrations/jdbc.md). | +| [odbc](odbc.md) | Создаёт таблицу с движком [ODBC](../../engines/table-engines/integrations/odbc.md). | +| [hdfs](hdfs.md) | Создаёт таблицу с движком [HDFS](../../engines/table-engines/integrations/hdfs.md). | +| [postgresql](postgresql.md) | Создает таблицу с движком [PostgreSQL](../../engines/table-engines/integrations/postgresql.md). | -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/) +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/) diff --git a/docs/ru/sql-reference/table-functions/postgresql.md b/docs/ru/sql-reference/table-functions/postgresql.md new file mode 100644 index 00000000000..e708e8304a5 --- /dev/null +++ b/docs/ru/sql-reference/table-functions/postgresql.md @@ -0,0 +1,73 @@ +--- +toc_priority: 54 +toc_title: postgresql +--- + +# postgresql {#postgresql} + +Позволяет выполнять заполсы `SELECT` и `INSERT` над таблицами удаленной БД PostgreSQL. + +**Синтаксис** + +``` sql +postgresql('host:port', 'database', 'table', 'user', 'password') +``` + +**Аргументы** + +- `host:port` — адрес сервера PostgreSQL. +- `database` — имя удаленной БД. +- `table` — таблица удаленной БД. +- `user` — пользователь PostgreSQL. +- `password` — пароль пользователя. + +**Возвращаемое значение** + +Таблица с теми же столбцами, как ив исходной таблице PostgreSQL. + +!!! info "Замечения" + В запросах `INSERT` чтобы различать табличную функцию `postgresql(...)` от таблицы с таким именем и списком колонок, используйте ключевые слова `FUNCTION` или `TABLE FUNCTION`. См. примеры ниже. + +**Примеры** + +Рассмотрим таблицу в БД PostgreSQL: + +``` sql +postgre> CREATE TABLE IF NOT EXISTS test_table (a integer, b text, c integer) +postgre> INSERT INTO test_table (a, b, c) VALUES (1, 2, 3), (4, 5, 6) +``` + +Получение данных из ClickHouse: + +``` sql +SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` text +┌─a─┬─b─┬─c─┐ +│ 1 │ 2 │ 3 │ +│ 4 │ 5 │ 6 │ +└───┴───┴───┘ +``` + +Вставка данных в таблицу PostgreSQL из таблицы ClickHouse: + +```sql +INSERT INTO FUNCTION postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword') (a, b, c) VALUES (7, 8, 9); +SELECT * FROM postgresql('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` text +┌─a─┬─b─┬─c─┐ +│ 1 │ 2 │ 3 │ +│ 4 │ 5 │ 6 │ +│ 7 │ 8 │ 9 │ +└───┴───┴───┘ +``` + +**См. также** + +- [Табличный бвижок PostgreSQL](../../engines/table-engines/integrations/postgresql.md) +- [Пример подключения PostgreSQL как источника внешнего словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/postgresql/) From bedb7c03764919442d1aff72c3be8eb6f7ab8695 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 14 Mar 2021 20:25:32 +0300 Subject: [PATCH 034/266] Some fixes --- docs/README.md | 10 ++- docs/ru/engines/database-engines/index.md | 2 - .../table-engines/integrations/postgresql.md | 77 +++++++++++++++++++ 3 files changed, 85 insertions(+), 4 deletions(-) create mode 100644 docs/ru/engines/table-engines/integrations/postgresql.md diff --git a/docs/README.md b/docs/README.md index a05f7df8968..a4df023a6ad 100644 --- a/docs/README.md +++ b/docs/README.md @@ -122,11 +122,17 @@ Contribute all new information in English language. Other languages are translat | Cell C1 | Cell C2 | Cell C3 | ``` -{##
+ ### Adding a New File -When adding a new file: +When you add a new file, it should end with a link like: + +`[Original article](https://clickhouse.tech/docs/) ` + +and there should be **a new empty line** after it. + +{## When adding a new file: - Make symbolic links for all other languages. You can use the following commands: diff --git a/docs/ru/engines/database-engines/index.md b/docs/ru/engines/database-engines/index.md index eee222d5f07..dbee7d4b658 100644 --- a/docs/ru/engines/database-engines/index.md +++ b/docs/ru/engines/database-engines/index.md @@ -18,8 +18,6 @@ toc_title: "\u0412\u0432\u0435\u0434\u0435\u043d\u0438\u0435" - [Lazy](../../engines/database-engines/lazy.md) -- [Atomic](../../engines/database-engines/atomic.md) - - [PostgreSQL](../../engines/database-engines/postgresql.md) [Оригинальная статья](https://clickhouse.tech/docs/ru/database_engines/) diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md new file mode 100644 index 00000000000..f6a357a140d --- /dev/null +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -0,0 +1,77 @@ +--- +toc_priority: 11 +toc_title: PostgreSQL +--- + +#PostgreSQL {#postgresql} + +Движок PostgreSQL позволяет выполнять запросы `SELECT` и `INSERT` для таблиц на удаленном сервере PostgreSQL. + +## Создание таблицы {#creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], + ... +) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); +``` + +См. подробное описание запроса [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query). + +**Параметры движка** + +- `host:port` — адрес сервера MySQL. + +- `database` — имя удаленной БД. + +- `table` — имя удаленной таблицы БД. + +- `user` — пользователь MySQL. + +- `password` — пароль пользователя. + +## Примеры использования {#usage-example} + +Рассмотрим таблицу ClickHouse, которая получает данные из таблицы PostgreSQL: + +``` sql +CREATE TABLE test_table +( + `int_id` Int32, + 'value' Int32 +) +ENGINE = PostgreSQL('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); +``` + +``` sql +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +└────────┴───────┘ +``` + +Добавление данных из таблицы ClickHouse в таблицу PosegreSQL: + +``` sql +INSERT INTO test_database.test_table VALUES (3,4); +SELECT * FROM test_database.test_table; +``` + +``` text +┌─int_id─┬─value─┐ +│ 1 │ 2 │ +│ 3 │ 4 │ +└────────┴───────┘ +``` + +## См. также {#see-also} + +- [Функция 'postgresql'](../../../sql-reference/table-functions/postgresql.md) +- [Пример подключения PostgreSQL как источника внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) + +[Оригинальная статья](https://clickhouse.tech/docs/en/operations/table-engines/integrations/postgresql/) From 49395cdd28cd93fbb2fa11c5e9e0c19faf10d86b Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 14 Mar 2021 20:48:23 +0300 Subject: [PATCH 035/266] Delete postgresql.md --- docs/ru/engines/table-engines/postgresql.md | 77 --------------------- 1 file changed, 77 deletions(-) delete mode 100644 docs/ru/engines/table-engines/postgresql.md diff --git a/docs/ru/engines/table-engines/postgresql.md b/docs/ru/engines/table-engines/postgresql.md deleted file mode 100644 index f6a357a140d..00000000000 --- a/docs/ru/engines/table-engines/postgresql.md +++ /dev/null @@ -1,77 +0,0 @@ ---- -toc_priority: 11 -toc_title: PostgreSQL ---- - -#PostgreSQL {#postgresql} - -Движок PostgreSQL позволяет выполнять запросы `SELECT` и `INSERT` для таблиц на удаленном сервере PostgreSQL. - -## Создание таблицы {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], - ... -) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); -``` - -См. подробное описание запроса [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query). - -**Параметры движка** - -- `host:port` — адрес сервера MySQL. - -- `database` — имя удаленной БД. - -- `table` — имя удаленной таблицы БД. - -- `user` — пользователь MySQL. - -- `password` — пароль пользователя. - -## Примеры использования {#usage-example} - -Рассмотрим таблицу ClickHouse, которая получает данные из таблицы PostgreSQL: - -``` sql -CREATE TABLE test_table -( - `int_id` Int32, - 'value' Int32 -) -ENGINE = PostgreSQL('localhost:5432', 'test_database', 'test_table', 'postgres', 'mysecretpassword'); -``` - -``` sql -SELECT * FROM test_database.test_table; -``` - -``` text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -└────────┴───────┘ -``` - -Добавление данных из таблицы ClickHouse в таблицу PosegreSQL: - -``` sql -INSERT INTO test_database.test_table VALUES (3,4); -SELECT * FROM test_database.test_table; -``` - -``` text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -│ 3 │ 4 │ -└────────┴───────┘ -``` - -## См. также {#see-also} - -- [Функция 'postgresql'](../../../sql-reference/table-functions/postgresql.md) -- [Пример подключения PostgreSQL как источника внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) - -[Оригинальная статья](https://clickhouse.tech/docs/en/operations/table-engines/integrations/postgresql/) From 735acf27d87afa546c5e3b0265941280678adce8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Mar 2021 19:39:31 +0300 Subject: [PATCH 036/266] try run stateful tests --- docker/test/stateful/run.sh | 64 +++++++++++++------ docker/test/stateless/clusters2.xml | 22 ------- docker/test/stateless/enable_test_keeper1.xml | 38 ----------- .../process_functional_tests_result.py | 12 +++- docker/test/stateless/run.sh | 15 ++--- docker/test/stateless/use_test_keeper.xml | 16 ----- tests/clickhouse-test | 2 + tests/queries/skip_list.json | 4 +- 8 files changed, 68 insertions(+), 105 deletions(-) delete mode 100644 docker/test/stateless/clusters2.xml delete mode 100644 docker/test/stateless/enable_test_keeper1.xml delete mode 100644 docker/test/stateless/use_test_keeper.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 195cfbfa3d3..e2edb4a0238 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -14,14 +14,17 @@ dpkg -i package_folder/clickhouse-test_*.deb function start() { if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + # NOTE We run "clickhouse server" instead of "clickhouse-server" + # to make "pidof clickhouse-server" return single pid of the main instance. + # We wil run main instance using "service clickhouse-server start" + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ @@ -51,9 +54,8 @@ start /s3downloader --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" -clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" -clickhouse-client --query "CREATE DATABASE test" +clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" service clickhouse-server restart # Wait for server to start accepting connections @@ -63,24 +65,50 @@ for _ in {1..120}; do done clickhouse-client --query "SHOW TABLES FROM datasets" -clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" -clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" -clickhouse-client --query "SHOW TABLES FROM test" - -if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test ; then - SKIP_LIST_OPT="--use-skip-list" -fi - -# We can have several additional options so we path them as array because it's -# more idiologically correct. -read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - ADDITIONAL_OPTIONS+=('--replicated-database') + clickhouse-client --query "CREATE DATABASE test ON CLUSTER 'test_cluster_database_replicated' + ENGINE=Replicated('/test/clickhouse/db/test', '{shard}', '{replica}')" + + clickhouse-client --query "CREATE TABLE test.hits AS datasets.hits_v1" + clickhouse-client --query "CREATE TABLE test.visits AS datasets.visits_v1" + + clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" + clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" + + clickhouse-client --query "DROP TABLE datasets.hits_v1" + clickhouse-client --query "DROP TABLE datasets.visits_v1" + + MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) + MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) +else + clickhouse-client --query "CREATE DATABASE test" + clickhouse-client --query "SHOW TABLES FROM test" + clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" + clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" fi -clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "SELECT count() FROM test.hits" +clickhouse-client --query "SELECT count() FROM test.visits" + +function run_tests() +{ + set -x + # We can have several additional options so we path them as array because it's + # more idiologically correct. + read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" + + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--replicated-database') + fi + + clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +} + +export -f run_tests +timeout "$MAX_RUN_TIME" bash -c run_tests ||: ./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv diff --git a/docker/test/stateless/clusters2.xml b/docker/test/stateless/clusters2.xml deleted file mode 100644 index 73b6274df12..00000000000 --- a/docker/test/stateless/clusters2.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - - localhost - 9000 - - - localhost - 19000 - - - - - localhost - 29000 - - - - - diff --git a/docker/test/stateless/enable_test_keeper1.xml b/docker/test/stateless/enable_test_keeper1.xml deleted file mode 100644 index 6c9669a42fd..00000000000 --- a/docker/test/stateless/enable_test_keeper1.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - 9181 - 1 - - - 5000 - 10000 - trace - - - - - 1 - localhost - 44444 - true - 3 - - - 2 - localhost - 44445 - true - true - 2 - - - 3 - localhost - 44446 - true - true - 1 - - - - diff --git a/docker/test/stateless/process_functional_tests_result.py b/docker/test/stateless/process_functional_tests_result.py index 27210ef9b80..02adf108212 100755 --- a/docker/test/stateless/process_functional_tests_result.py +++ b/docker/test/stateless/process_functional_tests_result.py @@ -12,6 +12,8 @@ UNKNOWN_SIGN = "[ UNKNOWN " SKIPPED_SIGN = "[ SKIPPED " HUNG_SIGN = "Found hung queries in processlist" +NO_TASK_TIMEOUT_SIGN = "All tests have finished" + def process_test_log(log_path): total = 0 skipped = 0 @@ -19,10 +21,13 @@ def process_test_log(log_path): failed = 0 success = 0 hung = False + task_timeout = True test_results = [] with open(log_path, 'r') as test_file: for line in test_file: line = line.strip() + if NO_TASK_TIMEOUT_SIGN in line: + task_timeout = False if HUNG_SIGN in line: hung = True if any(sign in line for sign in (OK_SIGN, FAIL_SING, UNKNOWN_SIGN, SKIPPED_SIGN)): @@ -52,7 +57,7 @@ def process_test_log(log_path): else: success += int(OK_SIGN in line) test_results.append((test_name, "OK", test_time)) - return total, skipped, unknown, failed, success, hung, test_results + return total, skipped, unknown, failed, success, hung, task_timeout, test_results def process_result(result_path): test_results = [] @@ -68,7 +73,7 @@ def process_result(result_path): state = "error" if result_path and os.path.exists(result_path): - total, skipped, unknown, failed, success, hung, test_results = process_test_log(result_path) + total, skipped, unknown, failed, success, hung, task_timeout, test_results = process_test_log(result_path) is_flacky_check = 1 < int(os.environ.get('NUM_TRIES', 1)) # If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately) # But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped. @@ -78,6 +83,9 @@ def process_result(result_path): if hung: description = "Some queries hung, " state = "failure" + elif task_timeout: + description = "Timeout, " + state = "failure" else: description = "" diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index dfa05e1a354..494be95b490 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -40,7 +40,7 @@ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ @@ -48,23 +48,23 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ --macros.shard s2 # It doesn't work :( + + MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) + MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi sleep 5 -if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then - SKIP_LIST_OPT="--use-skip-list" -fi - function run_tests() { + set -x # We can have several additional options so we path them as array because it's # more idiologically correct. read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" @@ -82,8 +82,7 @@ function run_tests() fi clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ - --test-runs "$NUM_TRIES" \ - "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt } diff --git a/docker/test/stateless/use_test_keeper.xml b/docker/test/stateless/use_test_keeper.xml deleted file mode 100644 index 26ec47e50e8..00000000000 --- a/docker/test/stateless/use_test_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - localhost - 9181 - - - localhost - 19181 - - - localhost - 29181 - - - diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fce8fe65f30..56e311a5e11 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -843,6 +843,8 @@ def main(args): if total_tests_run == 0: print("No tests were run.") sys.exit(1) + else: + print("All tests have finished.") sys.exit(exit_code) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 957108cc9cc..4b613369cc2 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -135,6 +135,7 @@ "00626_replace_partition_from_table_zookeeper", "00626_replace_partition_from_table", "00152_insert_different_granularity", + "00054_merge_tree_partitions", /// Old syntax is not allowed "01062_alter_on_mutataion_zookeeper", "00925_zookeeper_empty_replicated_merge_tree_optimize_final", @@ -150,7 +151,8 @@ "00083_create_merge_tree_zookeeper", "00062_replicated_merge_tree_alter_zookeeper", /// Does not support renaming of multiple tables in single query - "00634_rename_view" + "00634_rename_view", + "00140_rename" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From ff79be8af90e57feaf3dfb8d6c017349cbeb9403 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Mar 2021 23:01:20 +0300 Subject: [PATCH 037/266] fix --- docker/test/stateful/run.sh | 2 +- docker/test/stateless/run.sh | 2 +- src/Common/Macros.cpp | 3 + src/Common/Macros.h | 1 + src/Databases/DatabaseReplicated.cpp | 72 ++++++++++++++++--- src/Databases/DatabaseReplicated.h | 2 + tests/config/config.d/database_replicated.xml | 1 + 7 files changed, 73 insertions(+), 10 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index e2edb4a0238..3f840d9bfec 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -103,7 +103,7 @@ function run_tests() ADDITIONAL_OPTIONS+=('--replicated-database') fi - clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --use-skip-list --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 494be95b490..df8b78fddc0 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -82,7 +82,7 @@ function run_tests() fi clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ - --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --use-skip-list --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt } diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index b8e25499c0b..7882449b595 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -78,7 +78,10 @@ String Macros::expand(const String & s, /// Prefer explicit macros over implicit. if (it != macros.end() && !info.expand_special_macros_only) + { res += it->second; + info.expanded_other = true; + } else if (macro_name == "database" && !info.table_id.database_name.empty()) { res += info.table_id.database_name; diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 3082452e297..9298dbfc2d5 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -40,6 +40,7 @@ public: bool expanded_database = false; bool expanded_table = false; bool expanded_uuid = false; + bool expanded_other = false; bool has_unknown = false; }; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 83609606bb8..61eabf6ebea 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace DB { @@ -314,20 +315,65 @@ void DatabaseReplicated::loadStoredObjects(Context & context, bool has_force_res ddl_worker->startup(); } -BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context) +void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & query_context) const { - if (is_readonly) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper"); - - if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) - throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database."); - /// Replicas will set correct name of current database in query context (database name can be different on replicas) - if (auto * ddl_query = query->as()) + if (auto * ddl_query = dynamic_cast(query.get())) { if (ddl_query->database != getDatabaseName()) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed"); ddl_query->database.clear(); + + if (auto * create = query->as()) + { + bool replicated_table = create->storage && create->storage->engine && startsWith(create->storage->engine->name, "Replicated"); + if (!replicated_table || !create->storage->engine->arguments) + return; + + ASTs & args = create->storage->engine->arguments->children; + if (args.size() < 2) + return; + + ASTLiteral * arg1 = args[0]->as(); + ASTLiteral * arg2 = args[1]->as(); + if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) + return; + + String maybe_path = arg1->value.get(); + String maybe_replica = arg2->value.get(); + + /// Looks like it's ReplicatedMergeTree with explicit zookeeper_path and replica_name arguments. + /// Let's ensure that some macros are used. + /// NOTE: we cannot check here that substituted values will be actually different on shards and replicas. + + Macros::MacroExpansionInfo info; + info.table_id = {getDatabaseName(), create->table, create->uuid}; + query_context.getMacros()->expand(maybe_path, info); + bool maybe_shard_macros = info.expanded_other; + info.expanded_other = false; + query_context.getMacros()->expand(maybe_replica, info); + bool maybe_replica_macros = info.expanded_other; + bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); + + if (enable_functional_tests_helper) + { + if (maybe_path.empty() || maybe_path.back() != '/') + maybe_path += '/'; + arg1->value = maybe_path + "{shard}"; + arg2->value = maybe_replica + "{replica}"; + return; + } + + LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + + if (maybe_shard_macros && maybe_replica_macros) + return; + + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. " + "If you really want to specify it explicitly, then you should use some macros " + "to distinguish different shards and replicas"); + } } if (const auto * query_alter = query->as()) @@ -348,7 +394,17 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const "Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA or set " "database_replicated_always_detach_permanently to 1"); } +} +BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context) +{ + if (is_readonly) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper"); + + if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) + throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database."); + + checkQueryValid(query, query_context); LOG_DEBUG(log, "Proposing query: {}", queryToString(query)); DDLLogEntry entry; diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8f2ccd27627..46bcdc71ef1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -71,6 +71,8 @@ private: bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); + void checkQueryValid(const ASTPtr & query, const Context & query_context) const; + void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr); std::map tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr); diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 5b09a13ca4b..a030dd1e861 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -72,4 +72,5 @@ + <_functional_tests_helper_database_replicated_replace_args_macros>1 From 69cbb99f2c651d01ff352a1b5ce822c0427ea755 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Mar 2021 01:01:48 +0300 Subject: [PATCH 038/266] fix --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Interpreters/InterpreterDropQuery.cpp | 8 ++++---- src/Interpreters/InterpreterDropQuery.h | 4 ++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 37eaecf9a90..a4672887eac 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -48,6 +48,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); + query_ptr->as().database = table_id.database_name;; DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b30996b1dbf..276f61d244a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -82,7 +82,7 @@ void InterpreterDropQuery::waitForTableToBeActuallyDroppedOrDetached(const ASTDr db->waitDetachedTableNotInUse(uuid_to_wait); } -BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query) +BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query) { DatabasePtr database; UUID table_to_wait_on = UUIDHelpers::Nil; @@ -92,7 +92,7 @@ BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query) return res; } -BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) +BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) { /// NOTE: it does not contain UUID, we will resolve it with locked DDLGuard auto table_id = StorageID(query); @@ -101,7 +101,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat if (context.tryResolveStorageID(table_id, Context::ResolveExternal)) return executeToTemporaryTable(table_id.getTableName(), query.kind); else - table_id.database_name = context.getCurrentDatabase(); + query.database = table_id.database_name = context.getCurrentDatabase(); } if (query.temporary) @@ -212,7 +212,7 @@ BlockIO InterpreterDropQuery::executeToDictionary( String database_name = context.resolveDatabase(database_name_); auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name) : nullptr); - + query_ptr->as()->database = database_name; DatabasePtr database = tryGetDatabase(database_name, if_exists); bool is_drop_or_detach_database = query_ptr->as()->table.empty(); diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index d51ce3293ec..4a67857767f 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -34,8 +34,8 @@ private: BlockIO executeToDatabase(const ASTDropQuery & query); BlockIO executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait); - BlockIO executeToTable(const ASTDropQuery & query); - BlockIO executeToTableImpl(const ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); + BlockIO executeToTable(ASTDropQuery & query); + BlockIO executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); static void waitForTableToBeActuallyDroppedOrDetached(const ASTDropQuery & query, const DatabasePtr & db, const UUID & uuid_to_wait); From 7ae5e51ee9a65cbeefcda2d57876228a405d03c8 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 17 Mar 2021 01:50:37 +0300 Subject: [PATCH 039/266] Update InterpreterAlterQuery.cpp --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index a4672887eac..3ad7702548e 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -48,7 +48,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); - query_ptr->as().database = table_id.database_name;; + query_ptr->as().database = table_id.database_name; DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) From 2ccdb7ef5c5d8e7f5bd04f54d74209ad7184c90e Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 15 Mar 2021 18:44:15 +0300 Subject: [PATCH 040/266] Multiple small code and test updates - Updated the docs to make everything clear. - Multiple small logger fixes. - Changed the attach_part command -- now it's after check for the covering parts -- motivation is to do less work with the checksums fetching. - Better logging in the integration test. --- .../statements/alter/partition.md | 6 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 5 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 3 + .../MergeTree/ReplicatedMergeTreeLogEntry.h | 23 -- ...ReplicatedMergeTreeLogEntry_BACKUP_14564.h | 229 ------------------ .../ReplicatedMergeTreeLogEntry_BASE_14564.h | 200 --------------- .../ReplicatedMergeTreeLogEntry_LOCAL_14564.h | 205 ---------------- ...ReplicatedMergeTreeLogEntry_REMOTE_14564.h | 201 --------------- src/Storages/StorageReplicatedMergeTree.cpp | 51 ++-- .../test_attach_without_fetching/test.py | 38 +-- 11 files changed, 58 insertions(+), 905 deletions(-) delete mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BACKUP_14564.h delete mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BASE_14564.h delete mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_LOCAL_14564.h delete mode 100644 src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_REMOTE_14564.h diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index dd24231ca3f..96b48f7e737 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -61,9 +61,6 @@ ALTER TABLE mt DROP PARTITION '2020-11-21'; ALTER TABLE mt DROP PART 'all_4_4_0'; ``` -Note: the command does NOT throw an exception if the specified part does not exist, -e.g. `ALTER TABLE mt DROP PART 'i_do_not_exist'` will succeed. - ## DROP DETACHED PARTITION\|PART {#alter_drop-detached} ``` sql @@ -107,7 +104,8 @@ table on all replicas. ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 ``` -This query copies the data partition from the `table1` to `table2`. Note that data won’t be deleted from `table1`. +This query copies the data partition from the `table1` to `table2`. +Note that data won't be deleted neither from `table1` nor from `table2`. For the query to run successfully, the following conditions must be met: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 50db5754c60..ec03168d565 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2238,7 +2238,7 @@ 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 forgiving it.", part_to_detach->relative_path, prefix, part_to_detach->name); + 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/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index dd141a68248..01535ab0dbc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -293,11 +293,8 @@ String MergeTreeDataPartChecksums::getTotalChecksumHex() const { SipHash hash_of_all_files; - for (const auto & elem : files) + for (auto && [name, checksum] : files) { - const String & name = elem.first; - const auto & checksum = elem.second; - updateHash(hash_of_all_files, name); hash_of_all_files.update(checksum.file_hash); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index c91fa3a86d0..6f75b95cb42 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -270,6 +270,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// the MutableDataPartPtr here, we already have the data thus being able to /// calculate the checksums. log_entry.part_checksum = part->checksums.getTotalChecksumHex(); + + for (auto && [name, checksum] : part->checksums.files) + LOG_TRACE(log, "> On created file {}, file size {}", name, checksum.file_size); } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 129d7bf236a..309120560e7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -30,7 +30,6 @@ struct ReplicatedMergeTreeLogEntryData { enum Type { -<<<<<<< HEAD EMPTY, /// Not used. GET_PART, /// Get the part from another replica. ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). @@ -42,24 +41,12 @@ struct ReplicatedMergeTreeLogEntryData REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones MUTATE_PART, /// Apply one or several mutations to the part. ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths -======= - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths ->>>>>>> upstream/master }; static String typeToString(Type type) { switch (type) { -<<<<<<< HEAD case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; @@ -69,16 +56,6 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; -======= - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; ->>>>>>> upstream/master default: throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BACKUP_14564.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BACKUP_14564.h deleted file mode 100644 index 129d7bf236a..00000000000 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BACKUP_14564.h +++ /dev/null @@ -1,229 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -class ReadBuffer; -class WriteBuffer; -class ReplicatedMergeTreeQueue; - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -/// Record about what needs to be done. Only data (you can copy them). -struct ReplicatedMergeTreeLogEntryData -{ - enum Type - { -<<<<<<< HEAD - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). - /// You may think of it as a GET_PART with some optimisations as they're nearly identical. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths -======= - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths ->>>>>>> upstream/master - }; - - static String typeToString(Type type) - { - switch (type) - { -<<<<<<< HEAD - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; -======= - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; ->>>>>>> upstream/master - default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); - } - } - - String typeToString() const - { - return typeToString(type); - } - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - String toString() const; - - String znode_name; - - Type type = EMPTY; - String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. - - String part_checksum; /// Part checksum for ATTACH_PART, empty otherwise. - - /// The name of resulting part for GET_PART and MERGE_PARTS - /// Part range for DROP_RANGE and CLEAR_COLUMN - String new_part_name; - MergeTreeDataPartType new_part_type; - String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). - mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. - UUID new_part_uuid = UUIDHelpers::Nil; - - Strings source_parts; - bool deduplicate = false; /// Do deduplicate on merge - Strings deduplicate_by_columns = {}; // Which columns should be checked for duplicates, empty means 'all' (default). - MergeType merge_type = MergeType::REGULAR; - String column_name; - String index_name; - - /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. - bool detach = false; - - /// REPLACE PARTITION FROM command - struct ReplaceRangeEntry - { - String drop_range_part_name; - - String from_database; - String from_table; - Strings src_part_names; // as in from_table - Strings new_part_names; - Strings part_names_checksums; - int columns_version; - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - }; - - std::shared_ptr replace_range_entry; - - /// ALTER METADATA and MUTATE PART command - - /// Version of metadata which will be set after this alter - /// Also present in MUTATE_PART command, to track mutations - /// required for complete alter execution. - int alter_version = -1; /// May be equal to -1, if it's normal mutation, not metadata update. - - /// only ALTER METADATA command - bool have_mutation = false; /// If this alter requires additional mutation step, for data update - - String columns_str; /// New columns data corresponding to alter_version - String metadata_str; /// New metadata corresponding to alter_version - - /// Returns a set of parts that will appear after executing the entry + parts to block - /// selection of merges. These parts are added to queue.virtual_parts. - Strings getVirtualPartNames() const - { - /// Doesn't produce any part - if (type == ALTER_METADATA) - return {}; - - /// DROP_RANGE does not add a real part, but we must disable merges in that range - if (type == DROP_RANGE) - return {new_part_name}; - - /// Return {} because selection of merges in the partition where the column is cleared - /// should not be blocked (only execution of merges should be blocked). - if (type == CLEAR_COLUMN || type == CLEAR_INDEX) - return {}; - - if (type == REPLACE_RANGE) - { - Strings res = replace_range_entry->new_part_names; - res.emplace_back(replace_range_entry->drop_range_part_name); - return res; - } - - return {new_part_name}; - } - - /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. - /// These parts are added to future_parts. - Strings getBlockingPartNames() const - { - Strings res = getVirtualPartNames(); - - if (type == CLEAR_COLUMN) - res.emplace_back(new_part_name); - - return res; - } - - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - bool currently_executing = false; /// Whether the action is executing now. - bool removed_by_other_entry = false; - /// These several fields are informational only (for viewing by the user using system tables). - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). - std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. - time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. - size_t num_postponed = 0; /// The number of times the action was postponed. - String postpone_reason; /// The reason why the action was postponed, if it was postponed. - time_t last_postpone_time = 0; /// The time of the last time the action was postponed. - - /// Creation time or the time to copy from the general log to the queue of a particular replica. - time_t create_time = 0; - - /// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled. - size_t quorum = 0; - - /// If this MUTATE_PART entry caused by alter(modify/drop) query. - bool isAlterMutation() const - { - return type == MUTATE_PART && alter_version != -1; - } -}; - - -struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std::enable_shared_from_this -{ - using Ptr = std::shared_ptr; - - std::condition_variable execution_complete; /// Awake when currently_executing becomes false. - - static Ptr parse(const String & s, const Coordination::Stat & stat); -}; - -using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr; - - -} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BASE_14564.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BASE_14564.h deleted file mode 100644 index cdf5a40d5a9..00000000000 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_BASE_14564.h +++ /dev/null @@ -1,200 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -class ReadBuffer; -class WriteBuffer; -class ReplicatedMergeTreeQueue; - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -/// Record about what needs to be done. Only data (you can copy them). -struct ReplicatedMergeTreeLogEntryData -{ - enum Type - { - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths - }; - - static String typeToString(Type type) - { - switch (type) - { - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; - default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); - } - } - - String typeToString() const - { - return typeToString(type); - } - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - String toString() const; - - String znode_name; - - Type type = EMPTY; - String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. - - /// The name of resulting part for GET_PART and MERGE_PARTS - /// Part range for DROP_RANGE and CLEAR_COLUMN - String new_part_name; - MergeTreeDataPartType new_part_type; - String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). - mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. - UUID new_part_uuid = UUIDHelpers::Nil; - - Strings source_parts; - bool deduplicate = false; /// Do deduplicate on merge - Strings deduplicate_by_columns = {}; // Which columns should be checked for duplicates, empty means 'all' (default). - MergeType merge_type = MergeType::REGULAR; - String column_name; - String index_name; - - /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. - bool detach = false; - - /// REPLACE PARTITION FROM command - struct ReplaceRangeEntry - { - String drop_range_part_name; - - String from_database; - String from_table; - Strings src_part_names; // as in from_table - Strings new_part_names; - Strings part_names_checksums; - int columns_version; - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - }; - - std::shared_ptr replace_range_entry; - - /// ALTER METADATA and MUTATE PART command - - /// Version of metadata which will be set after this alter - /// Also present in MUTATE_PART command, to track mutations - /// required for complete alter execution. - int alter_version = -1; /// May be equal to -1, if it's normal mutation, not metadata update. - - /// only ALTER METADATA command - bool have_mutation = false; /// If this alter requires additional mutation step, for data update - - String columns_str; /// New columns data corresponding to alter_version - String metadata_str; /// New metadata corresponding to alter_version - - /// Returns a set of parts that will appear after executing the entry + parts to block - /// selection of merges. These parts are added to queue.virtual_parts. - Strings getVirtualPartNames() const - { - /// Doesn't produce any part - if (type == ALTER_METADATA) - return {}; - - /// DROP_RANGE does not add a real part, but we must disable merges in that range - if (type == DROP_RANGE) - return {new_part_name}; - - /// Return {} because selection of merges in the partition where the column is cleared - /// should not be blocked (only execution of merges should be blocked). - if (type == CLEAR_COLUMN || type == CLEAR_INDEX) - return {}; - - if (type == REPLACE_RANGE) - { - Strings res = replace_range_entry->new_part_names; - res.emplace_back(replace_range_entry->drop_range_part_name); - return res; - } - - return {new_part_name}; - } - - /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. - /// These parts are added to future_parts. - Strings getBlockingPartNames() const - { - Strings res = getVirtualPartNames(); - - if (type == CLEAR_COLUMN) - res.emplace_back(new_part_name); - - return res; - } - - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - bool currently_executing = false; /// Whether the action is executing now. - bool removed_by_other_entry = false; - /// These several fields are informational only (for viewing by the user using system tables). - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). - std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. - time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. - size_t num_postponed = 0; /// The number of times the action was postponed. - String postpone_reason; /// The reason why the action was postponed, if it was postponed. - time_t last_postpone_time = 0; /// The time of the last time the action was postponed. - - /// Creation time or the time to copy from the general log to the queue of a particular replica. - time_t create_time = 0; - - /// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled. - size_t quorum = 0; - - /// If this MUTATE_PART entry caused by alter(modify/drop) query. - bool isAlterMutation() const - { - return type == MUTATE_PART && alter_version != -1; - } -}; - - -struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std::enable_shared_from_this -{ - using Ptr = std::shared_ptr; - - std::condition_variable execution_complete; /// Awake when currently_executing becomes false. - - static Ptr parse(const String & s, const Coordination::Stat & stat); -}; - -using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr; - - -} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_LOCAL_14564.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_LOCAL_14564.h deleted file mode 100644 index b737bffeb28..00000000000 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_LOCAL_14564.h +++ /dev/null @@ -1,205 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -class ReadBuffer; -class WriteBuffer; -class ReplicatedMergeTreeQueue; - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -/// Record about what needs to be done. Only data (you can copy them). -struct ReplicatedMergeTreeLogEntryData -{ - enum Type - { - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - ATTACH_PART, /// Attach the part, possibly from our own replica (if found in /detached folder). - /// You may think of it as a GET_PART with some optimisations as they're nearly identical. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths - }; - - static String typeToString(Type type) - { - switch (type) - { - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; - default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); - } - } - - String typeToString() const - { - return typeToString(type); - } - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - String toString() const; - - String znode_name; - - Type type = EMPTY; - String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. - - String part_checksum; /// Part checksum for ATTACH_PART, empty otherwise. - - /// The name of resulting part for GET_PART and MERGE_PARTS - /// Part range for DROP_RANGE and CLEAR_COLUMN - String new_part_name; - MergeTreeDataPartType new_part_type; - String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). - mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. - UUID new_part_uuid = UUIDHelpers::Nil; - - Strings source_parts; - bool deduplicate = false; /// Do deduplicate on merge - Strings deduplicate_by_columns = {}; // Which columns should be checked for duplicates, empty means 'all' (default). - MergeType merge_type = MergeType::REGULAR; - String column_name; - String index_name; - - /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. - bool detach = false; - - /// REPLACE PARTITION FROM command - struct ReplaceRangeEntry - { - String drop_range_part_name; - - String from_database; - String from_table; - Strings src_part_names; // as in from_table - Strings new_part_names; - Strings part_names_checksums; - int columns_version; - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - }; - - std::shared_ptr replace_range_entry; - - /// ALTER METADATA and MUTATE PART command - - /// Version of metadata which will be set after this alter - /// Also present in MUTATE_PART command, to track mutations - /// required for complete alter execution. - int alter_version = -1; /// May be equal to -1, if it's normal mutation, not metadata update. - - /// only ALTER METADATA command - bool have_mutation = false; /// If this alter requires additional mutation step, for data update - - String columns_str; /// New columns data corresponding to alter_version - String metadata_str; /// New metadata corresponding to alter_version - - /// Returns a set of parts that will appear after executing the entry + parts to block - /// selection of merges. These parts are added to queue.virtual_parts. - Strings getVirtualPartNames() const - { - /// Doesn't produce any part - if (type == ALTER_METADATA) - return {}; - - /// DROP_RANGE does not add a real part, but we must disable merges in that range - if (type == DROP_RANGE) - return {new_part_name}; - - /// Return {} because selection of merges in the partition where the column is cleared - /// should not be blocked (only execution of merges should be blocked). - if (type == CLEAR_COLUMN || type == CLEAR_INDEX) - return {}; - - if (type == REPLACE_RANGE) - { - Strings res = replace_range_entry->new_part_names; - res.emplace_back(replace_range_entry->drop_range_part_name); - return res; - } - - return {new_part_name}; - } - - /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. - /// These parts are added to future_parts. - Strings getBlockingPartNames() const - { - Strings res = getVirtualPartNames(); - - if (type == CLEAR_COLUMN) - res.emplace_back(new_part_name); - - return res; - } - - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - bool currently_executing = false; /// Whether the action is executing now. - bool removed_by_other_entry = false; - /// These several fields are informational only (for viewing by the user using system tables). - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). - std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. - time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. - size_t num_postponed = 0; /// The number of times the action was postponed. - String postpone_reason; /// The reason why the action was postponed, if it was postponed. - time_t last_postpone_time = 0; /// The time of the last time the action was postponed. - - /// Creation time or the time to copy from the general log to the queue of a particular replica. - time_t create_time = 0; - - /// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled. - size_t quorum = 0; - - /// If this MUTATE_PART entry caused by alter(modify/drop) query. - bool isAlterMutation() const - { - return type == MUTATE_PART && alter_version != -1; - } -}; - - -struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std::enable_shared_from_this -{ - using Ptr = std::shared_ptr; - - std::condition_variable execution_complete; /// Awake when currently_executing becomes false. - - static Ptr parse(const String & s, const Coordination::Stat & stat); -}; - -using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr; - - -} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_REMOTE_14564.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_REMOTE_14564.h deleted file mode 100644 index afd8c963943..00000000000 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry_REMOTE_14564.h +++ /dev/null @@ -1,201 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -class ReadBuffer; -class WriteBuffer; -class ReplicatedMergeTreeQueue; - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -/// Record about what needs to be done. Only data (you can copy them). -struct ReplicatedMergeTreeLogEntryData -{ - enum Type - { - EMPTY, /// Not used. - GET_PART, /// Get the part from another replica. - MERGE_PARTS, /// Merge the parts. - DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. - CLEAR_COLUMN, /// NOTE: Deprecated. Drop specific column from specified partition. - CLEAR_INDEX, /// NOTE: Deprecated. Drop specific index from specified partition. - REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones - MUTATE_PART, /// Apply one or several mutations to the part. - ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths - }; - - static String typeToString(Type type) - { - switch (type) - { - case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; - case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; - case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; - case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; - case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; - case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; - case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; - case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA"; - default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); - } - } - - String typeToString() const - { - return typeToString(type); - } - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - String toString() const; - - String znode_name; - - Type type = EMPTY; - String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. - - /// The name of resulting part for GET_PART and MERGE_PARTS - /// Part range for DROP_RANGE and CLEAR_COLUMN - String new_part_name; - MergeTreeDataPartType new_part_type; - String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). - mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. - UUID new_part_uuid = UUIDHelpers::Nil; - - Strings source_parts; - bool deduplicate = false; /// Do deduplicate on merge - Strings deduplicate_by_columns = {}; // Which columns should be checked for duplicates, empty means 'all' (default). - MergeType merge_type = MergeType::REGULAR; - String column_name; - String index_name; - - /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. - bool detach = false; - - /// REPLACE PARTITION FROM command - struct ReplaceRangeEntry - { - String drop_range_part_name; - - String from_database; - String from_table; - Strings src_part_names; // as in from_table - Strings new_part_names; - Strings part_names_checksums; - int columns_version; - - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); - }; - - std::shared_ptr replace_range_entry; - - /// ALTER METADATA and MUTATE PART command - - /// Version of metadata which will be set after this alter - /// Also present in MUTATE_PART command, to track mutations - /// required for complete alter execution. - int alter_version = -1; /// May be equal to -1, if it's normal mutation, not metadata update. - - /// only ALTER METADATA command - bool have_mutation = false; /// If this alter requires additional mutation step, for data update - - String columns_str; /// New columns data corresponding to alter_version - String metadata_str; /// New metadata corresponding to alter_version - - /// Returns a set of parts that will appear after executing the entry + parts to block - /// selection of merges. These parts are added to queue.virtual_parts. - Strings getVirtualPartNames() const - { - /// Doesn't produce any part - if (type == ALTER_METADATA) - return {}; - - /// DROP_RANGE does not add a real part, but we must disable merges in that range - if (type == DROP_RANGE) - return {new_part_name}; - - /// Return {} because selection of merges in the partition where the column is cleared - /// should not be blocked (only execution of merges should be blocked). - if (type == CLEAR_COLUMN || type == CLEAR_INDEX) - return {}; - - if (type == REPLACE_RANGE) - { - Strings res = replace_range_entry->new_part_names; - res.emplace_back(replace_range_entry->drop_range_part_name); - return res; - } - - return {new_part_name}; - } - - /// Returns set of parts that denote the block number ranges that should be blocked during the entry execution. - /// These parts are added to future_parts. - Strings getBlockingPartNames() const - { - Strings res = getVirtualPartNames(); - - if (type == CLEAR_COLUMN) - res.emplace_back(new_part_name); - - return res; - } - - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - bool currently_executing = false; /// Whether the action is executing now. - bool removed_by_other_entry = false; - /// These several fields are informational only (for viewing by the user using system tables). - /// Access under queue_mutex, see ReplicatedMergeTreeQueue. - size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). - std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. - time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. - size_t num_postponed = 0; /// The number of times the action was postponed. - String postpone_reason; /// The reason why the action was postponed, if it was postponed. - time_t last_postpone_time = 0; /// The time of the last time the action was postponed. - - /// Creation time or the time to copy from the general log to the queue of a particular replica. - time_t create_time = 0; - - /// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled. - size_t quorum = 0; - - /// If this MUTATE_PART entry caused by alter(modify/drop) query. - bool isAlterMutation() const - { - return type == MUTATE_PART && alter_version != -1; - } -}; - - -struct ReplicatedMergeTreeLogEntry : public ReplicatedMergeTreeLogEntryData, std::enable_shared_from_this -{ - using Ptr = std::shared_ptr; - - std::condition_variable execution_complete; /// Awake when currently_executing becomes false. - - static Ptr parse(const String & s, const Coordination::Stat & stat); -}; - -using ReplicatedMergeTreeLogEntryPtr = std::shared_ptr; - - -} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e46e8e1685c..0f507f4f689 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1381,7 +1381,14 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo MergeTreeData::MutableDataPartPtr iter_part_ptr = createPart(part_name, part_iter, single_disk_volume, part_to_path); - if (part_checksum != iter_part_ptr->checksums.getTotalChecksumHex()) + const String iter_part_checksums = iter_part_ptr->checksums.getTotalChecksumHex(); + + LOG_TRACE(log, "Candidate part: {}, path: {}, checksums: {}", part_name, part_to_path, iter_part_checksums); + + for (auto && [name, checksum] : iter_part_ptr->checksums.files) + LOG_TRACE(log, "> File {}, file size {}", name, checksum.file_size); + + if (part_checksum != iter_part_checksums) continue; return iter_part_ptr; @@ -1405,19 +1412,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } - if (entry.type == LogEntry::ATTACH_PART) - { - if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) - { - Transaction transaction(*this); - - if (renameTempPartAndAdd(part, nullptr, &transaction)) - checkPartChecksumsAndCommit(transaction, part); - - return true; - } - } - const bool is_get_or_attach = entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART; if (is_get_or_attach || entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) @@ -1441,6 +1435,27 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) } } + if (entry.type == LogEntry::ATTACH_PART) + { + LOG_TRACE(log, "Trying to find part in detached/"); + + if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) + { + LOG_TRACE(log, "Found valid part {} to attach from local data, preparing the transaction", + part->name); + + Transaction transaction(*this); + + // don't need the replaced parts + renameTempPartAndReplace(part, nullptr, &transaction); + checkPartChecksumsAndCommit(transaction, part); + + return true; + } + + LOG_TRACE(log, "Didn't find part with the correct checksums, will fetch it from other replica"); + } + if (is_get_or_attach && entry.source_replica == replica_name) LOG_WARNING(log, "Part {} from own log doesn't exist.", entry.new_part_name); @@ -1831,18 +1846,10 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) const auto storage_settings_ptr = getSettings(); auto metadata_snapshot = getInMemoryMetadataPtr(); -<<<<<<< HEAD - static std::atomic_uint total_fetches {0}; - if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) throw Exception("Too many total fetches from replicas, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches.toString(), -======= - if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) - { - throw Exception("Too many total fetches from replicas, maximum: " + storage_settings_ptr->replicated_max_parallel_fetches.toString(), ->>>>>>> upstream/master ErrorCodes::TOO_MANY_FETCHES); ++total_fetches; diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index a9acad5a03b..3ab6fc3e525 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -38,22 +38,25 @@ def start_cluster(): def check_data(nodes, detached_parts): for node in nodes: + print("> Replication queue for", node.name, "\n> table\treplica_name\tsource_replica\ttype\tposition\n", + node.query("SELECT table, replica_name, source_replica, type, position FROM system.replication_queue")) + node.query("SYSTEM SYNC REPLICA test") + print("> Checking data integrity for", node.name) + for i in range(10): - assert node.query("SELECT count() FROM test WHERE n % 10 == " + str(i)) == \ - "0\n" if i in detached_parts else "10\n" + assert_eq_with_retry(node, "SELECT count() FROM test WHERE n % 10 == " + str(i), + "0\n" if i in detached_parts else "10\n") - assert node.query("SELECT count() FROM system.parts WHERE table='test'") == \ - str(10 - len(detached_parts)) + "\n" - - # We don't check for system.detached_parts count = len(detached parts) as the newly downloaded data - # is not removed from detached_parts (so this is wrong) + assert_eq_with_retry(node, "SELECT count() FROM system.parts WHERE table='test'", + str(10 - len(detached_parts)) + "\n") res: str = node.query("SELECT * FROM test ORDER BY n") for other in nodes: if other != node: + print("> Checking data consistency,", other.name, "vs", node.name) assert_eq_with_retry(other, "SELECT * FROM test ORDER BY n", res) @@ -82,9 +85,19 @@ def test_attach_without_fetching(start_cluster): # 2. Create the third replica fill_node(node_3) - # 3. Attach the first part and check if it has been fetched correctly. + # 3. Break the part data on the second node to corrupt the checksums. + # Replica 3 should download the data from replica 1 as there is no local data. + # Replica 2 should also download the data from 1 as the checksums won't match. + print("Checking attach with corrupted checksums") + corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") + node_1.query("ALTER TABLE test ATTACH PARTITION 1") + check_data([node_1, node_2, node_3], detached_parts=[0]) + + # 4. Attach the first part and check if it has been fetched correctly. # Replica 1 should attach the local data from detached/. # Replica 3 should download the data from replica 1 as there is no local data and other connections are broken. + print("Checking attach with valid checksums") + with PartitionManager() as pm: # If something goes wrong and replica 1 wants to fetch data, the test will fail. pm.partition_instances(node_1, node_2) @@ -92,11 +105,4 @@ def test_attach_without_fetching(start_cluster): node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") - check_data([node_1, node_2, node_3], detached_parts=[1]) - - # 4. Break the part data on the second node to corrupt the checksums. - # Replica 3 should download the data from replica 1 as there is no local data. - # Replica 2 should also download the data from 1 as the checksums won't match. - corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") - node_1.query("ALTER TABLE test ATTACH PARTITION 1") - check_data([node_1, node_2, node_3], detached_parts=[]) + check_data([node_1, node_2, node_3], detached_parts=[]) From 831b90f2725d47003b65abec1d8dcc579230a6e3 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 17 Mar 2021 00:59:12 +0300 Subject: [PATCH 041/266] Replaced the iteration algorithm in part finder Now, instead of iterating through the directories, we iterate though directories of on of the table disks (which doesn't give us a substantial boost but is a bit neater to read). - Updated the system.replication_queue command types. - Fixed the part ptr being empty (added the checksum loading and initialization). - Removed extra logging. --- .../system-tables/replication_queue.md | 12 ++++- .../statements/alter/partition.md | 3 -- src/Storages/MergeTree/MergeTreeData.cpp | 31 +++++++----- .../ReplicatedMergeTreeBlockOutputStream.cpp | 3 -- src/Storages/StorageReplicatedMergeTree.cpp | 47 +++++++------------ 5 files changed, 48 insertions(+), 48 deletions(-) diff --git a/docs/en/operations/system-tables/replication_queue.md b/docs/en/operations/system-tables/replication_queue.md index d1c74a771c6..f3e3a35f13b 100644 --- a/docs/en/operations/system-tables/replication_queue.md +++ b/docs/en/operations/system-tables/replication_queue.md @@ -14,7 +14,17 @@ Columns: - `node_name` ([String](../../sql-reference/data-types/string.md)) — Node name in ZooKeeper. -- `type` ([String](../../sql-reference/data-types/string.md)) — Type of the task in the queue: `GET_PARTS`, `MERGE_PARTS`, `DETACH_PARTS`, `DROP_PARTS`, or `MUTATE_PARTS`. +- `type` ([String](../../sql-reference/data-types/string.md)) — Type of the task in the queue, one of: + - `GET_PART` - Get the part from another replica. + - `ATTACH_PART` - Attach the part, possibly from our own replica (if found in `detached` folder). + You may think of it as a `GET_PART` with some optimisations as they're nearly identical. + - `MERGE_PARTS` - Merge the parts. + - `DROP_RANGE` - Delete the parts in the specified partition in the specified number range. + - `CLEAR_COLUMN` - NOTE: Deprecated. Drop specific column from specified partition. + - `CLEAR_INDEX` - NOTE: Deprecated. Drop specific index from specified partition. + - `REPLACE_RANGE` - Drop certain range of partitions and replace them by new ones + - `MUTATE_PART` - Apply one or several mutations to the part. + - `ALTER_METADATA` - Apply alter modification according to global /metadata and /columns paths - `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was submitted for execution. diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 96b48f7e737..f7183ba525c 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -70,9 +70,6 @@ ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr Removes the specified part or all parts of the specified partition from `detached`. Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). -Note: the command does NOT throw an exception if the specified part does not exist, -e.g. `ALTER TABLE mt DROP DETACHED PART[ITION] 'i_do_not_exist'` will succeed. - ## ATTACH PARTITION\|PART {#alter_attach-partition} ``` sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec03168d565..9ec26c21f4c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3160,15 +3160,18 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, const Cont MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, const Context & context, PartsTemporaryRename & renamed_parts) { - String source_dir = "detached/"; + const String source_dir = "detached/"; std::map name_to_disk; + /// Let's compose a list of parts that should be added. if (attach_part) { - String part_id = partition->as().value.safeGet(); + const String part_id = partition->as().value.safeGet(); + validateDetachedPartName(part_id); renamed_parts.addPart(part_id, "attaching_" + part_id); + if (MergeTreePartInfo::tryParsePartName(part_id, nullptr, format_version)) name_to_disk[part_id] = getDiskForPart(part_id, source_dir); } @@ -3179,12 +3182,14 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ActiveDataPartSet active_parts(format_version); const auto disks = getStoragePolicy()->getDisks(); + for (const auto & disk : disks) { for (auto it = disk->iterateDirectory(relative_data_path + source_dir); it->isValid(); it->next()) { const String & name = it->name(); MergeTreePartInfo part_info; + // TODO what if name contains "_tryN" suffix? /// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored if (!MergeTreePartInfo::tryParsePartName(name, &part_info, format_version) @@ -3192,21 +3197,23 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const { continue; } + LOG_DEBUG(log, "Found part {}", name); active_parts.add(name); name_to_disk[name] = disk; } } LOG_DEBUG(log, "{} of them are active", active_parts.size()); - /// Inactive parts rename so they can not be attached in case of repeated ATTACH. + + /// Inactive parts are renamed so they can not be attached in case of repeated ATTACH. for (const auto & [name, disk] : name_to_disk) { - String containing_part = active_parts.getContainingPart(name); + const String containing_part = active_parts.getContainingPart(name); + if (!containing_part.empty() && containing_part != name) - { // TODO maybe use PartsTemporaryRename here? - disk->moveDirectory(relative_data_path + source_dir + name, relative_data_path + source_dir + "inactive_" + name); - } + disk->moveDirectory(relative_data_path + source_dir + name, + relative_data_path + source_dir + "inactive_" + name); else renamed_parts.addPart(name, "attaching_" + name); } @@ -3221,11 +3228,13 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const MutableDataPartsVector loaded_parts; loaded_parts.reserve(renamed_parts.old_and_new_names.size()); - for (const auto & part_names : renamed_parts.old_and_new_names) + for (const auto & [old_name, new_name] : renamed_parts.old_and_new_names) { - LOG_DEBUG(log, "Checking part {}", part_names.second); - auto single_disk_volume = std::make_shared("volume_" + part_names.first, name_to_disk[part_names.first], 0); - MutableDataPartPtr part = createPart(part_names.first, single_disk_volume, source_dir + part_names.second); + LOG_DEBUG(log, "Checking part {}", new_name); + + auto single_disk_volume = std::make_shared("volume_" + old_name, name_to_disk[old_name]); + MutableDataPartPtr part = createPart(old_name, single_disk_volume, source_dir + new_name); + loadPartAndFixMetadataImpl(part); loaded_parts.push_back(part); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 6f75b95cb42..c91fa3a86d0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -270,9 +270,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// the MutableDataPartPtr here, we already have the data thus being able to /// calculate the checksums. log_entry.part_checksum = part->checksums.getTotalChecksumHex(); - - for (auto && [name, checksum] : part->checksums.files) - LOG_TRACE(log, "> On created file {}, file size {}", name, checksum.file_size); } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f507f4f689..6552a9bedf1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1357,43 +1357,33 @@ String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataP MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFoundValidPart(const LogEntry& entry) const { - const MergeTreePartInfo target_part = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); - const String& part_checksum = entry.part_checksum; + const MergeTreePartInfo actual_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); + const String part_new_name = actual_part_info.getPartName(); - Poco::DirectoryIterator dir_end; + LOG_TRACE(log, "Trying to attach part {} from local data", part_new_name); - for (const String& path : getDataPaths()) - { - for (Poco::DirectoryIterator it{path + "detached/"}; it != dir_end; ++it) + for (const DiskPtr & disk : getStoragePolicy()->getDisks()) + for (const auto it = disk->iterateDirectory(relative_data_path + "detached/"); it->isValid(); it->next()) { - MergeTreePartInfo part_iter; + MergeTreePartInfo part_info; - if (!MergeTreePartInfo::tryParsePartName(it.name(), &part_iter, format_version) || - part_iter.partition_id != target_part.partition_id) + if (!MergeTreePartInfo::tryParsePartName(it->name(), &part_info, format_version) || + part_info.partition_id != actual_part_info.partition_id) continue; - const String& part_name = part_iter.getPartName(); - const String part_to_path = "detached/" + part_name; + const String part_old_name = part_info.getPartName(); + const String part_path = "detached/" + part_old_name; - auto single_disk_volume = std::make_shared("volume_" + part_name, - getDiskForPart(part_name, "detached/")); + const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); + MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, part_info, volume, part_path); - MergeTreeData::MutableDataPartPtr iter_part_ptr = - createPart(part_name, part_iter, single_disk_volume, part_to_path); + // We don't check consistency as in that case this method will throw. + // The faster way is to load invalid data and just check the checksums -- they won't match. + part->loadColumnsChecksumsIndexes(true, false); - const String iter_part_checksums = iter_part_ptr->checksums.getTotalChecksumHex(); - - LOG_TRACE(log, "Candidate part: {}, path: {}, checksums: {}", part_name, part_to_path, iter_part_checksums); - - for (auto && [name, checksum] : iter_part_ptr->checksums.files) - LOG_TRACE(log, "> File {}, file size {}", name, checksum.file_size); - - if (part_checksum != iter_part_checksums) - continue; - - return iter_part_ptr; + if (entry.part_checksum == part->checksums.getTotalChecksumHex()) + return part; } - } return {}; } @@ -1437,8 +1427,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (entry.type == LogEntry::ATTACH_PART) { - LOG_TRACE(log, "Trying to find part in detached/"); - if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) { LOG_TRACE(log, "Found valid part {} to attach from local data, preparing the transaction", @@ -1446,7 +1434,6 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) Transaction transaction(*this); - // don't need the replaced parts renameTempPartAndReplace(part, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); From 91068b782bb9c5b0fe66408bcb8ae9014e1d26f0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Mar 2021 17:29:24 +0300 Subject: [PATCH 042/266] fix --- src/Interpreters/DDLTask.cpp | 11 +++++++++++ src/Interpreters/DDLTask.h | 3 ++- tests/integration/test_replicated_database/test.py | 11 ++++++++++- 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 1cfd113e81f..d446422124a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -344,6 +344,17 @@ String DatabaseReplicatedTask::getShardID() const return database->shard_name; } +void DatabaseReplicatedTask::parseQueryFromEntry(const Context & context) +{ + DDLTaskBase::parseQueryFromEntry(context); + if (auto * ddl_query = dynamic_cast(query.get())) + { + /// Update database name with actual name of local database + assert(ddl_query->database.empty()); + ddl_query->database = database->getDatabaseName(); + } +} + std::unique_ptr DatabaseReplicatedTask::makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) { auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper); diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index b794668f802..a333bb1af45 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -93,7 +93,7 @@ struct DDLTaskBase DDLTaskBase(const DDLTaskBase &) = delete; virtual ~DDLTaskBase() = default; - void parseQueryFromEntry(const Context & context); + virtual void parseQueryFromEntry(const Context & context); virtual String getShardID() const = 0; @@ -134,6 +134,7 @@ struct DatabaseReplicatedTask : public DDLTaskBase DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_); String getShardID() const override; + void parseQueryFromEntry(const Context & context) override; std::unique_ptr makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) override; DatabaseReplicated * database; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index f02457b144a..70d779ea737 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -35,8 +35,17 @@ def started_cluster(): cluster.shutdown() def test_create_replicated_table(started_cluster): + assert "Explicit zookeeper_path and replica_name are specified" in \ + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp', 'r') ORDER BY k PARTITION BY toYYYYMM(d);") + + assert "Explicit zookeeper_path and replica_name are specified" in \ + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp', 'r', d, k, 8192);") + assert "Old syntax is not allowed" in \ - main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/test/tmp', 'r', d, k, 8192);") + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp/{shard}', '{replica}', d, k, 8192);") main_node.query("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);") From e309dd275acccda76ae93af5737075069a6bdc9e Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 17 Mar 2021 18:49:04 +0300 Subject: [PATCH 043/266] Trying to investigate the errors after attaching the part. + A new entry is added to the part_log. --- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d1371609e1a..453ae513dc6 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -460,7 +460,7 @@ public: /// Remove parts from working set immediately (without wait for background /// process). Transfer part state to temporary. Have very limited usage only - /// for new parts which don't already present in table. + /// for new parts which aren't already present in table. void removePartsFromWorkingSetImmediatelyAndSetTemporaryState(const DataPartsVector & remove); /// Removes parts from the working set parts. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6552a9bedf1..46920258567 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1382,7 +1382,11 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo part->loadColumnsChecksumsIndexes(true, false); if (entry.part_checksum == part->checksums.getTotalChecksumHex()) + { + part->is_temp = true; + part->modification_time = disk->getLastModified(part_path).epochTime(); return part; + } } return {}; @@ -1437,6 +1441,12 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) renameTempPartAndReplace(part, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); + writePartLog(PartLogElement::Type::NEW_PART, {}, + 0, // well, not really, but don't have the idea how to measure the time here TODO + part->name, part, + {part}, // not sure whether the initial parts vector should be empty or contain the part itself TODO + nullptr); + return true; } @@ -3706,6 +3716,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora String interserver_scheme; std::optional tagger_ptr; std::function get_part; + if (part_to_clone) { get_part = [&, part_to_clone]() From 710c491f63bc24f26e266766a6b693e3c50fec19 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 17 Mar 2021 19:50:56 +0300 Subject: [PATCH 044/266] Update database_replicated.xml --- tests/config/config.d/database_replicated.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index a030dd1e861..3bbae6031c9 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -22,6 +22,7 @@ 5000 10000 trace + false From fdae70df6ca2e5ddd71d9cae39139af02f5acbd4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Mar 2021 15:49:31 +0300 Subject: [PATCH 045/266] fix tests --- docker/test/stateful/run.sh | 16 +++++++-------- docker/test/stateless/run.sh | 16 +++++++-------- src/Databases/DatabaseReplicated.cpp | 15 +++++++------- tests/clickhouse-test | 9 ++++++++- tests/config/install.sh | 20 +++++++++---------- ...ted_minimalistic_part_header_zookeeper.sql | 8 ++++---- ...0953_zookeeper_suetin_deduplication_bug.sh | 14 ++++++------- ...ion_collapsing_attach_detach_zookeeper.sql | 4 ++-- ...icated_mutations_empty_partition.reference | 2 +- ...6_replicated_mutations_empty_partition.sql | 6 +++--- ...ter_mutations_kill_many_replicas.reference | 10 +++++----- ...rent_alter_mutations_kill_many_replicas.sh | 8 +++++--- .../01700_system_zookeeper_path_in.reference | 6 ++---- .../01700_system_zookeeper_path_in.sql | 16 ++++++++------- tests/queries/skip_list.json | 8 ++++++++ 15 files changed, 88 insertions(+), 70 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3f840d9bfec..3682686bc99 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -17,16 +17,16 @@ function start() # NOTE We run "clickhouse server" instead of "clickhouse-server" # to make "pidof clickhouse-server" return single pid of the main instance. # We wil run main instance using "service clickhouse-server start" - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ - -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ + -- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ - -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 @@ -118,8 +118,8 @@ if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||: pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: - pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: - mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index df8b78fddc0..fda9e61dd29 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -40,17 +40,17 @@ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ - -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ + -- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ - -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ @@ -102,8 +102,8 @@ tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||: pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: - pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: - mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 61eabf6ebea..9ae03ad0900 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -355,20 +355,21 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q bool maybe_replica_macros = info.expanded_other; bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); + if (!enable_functional_tests_helper) + LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + + if (maybe_shard_macros && maybe_replica_macros) + return; + if (enable_functional_tests_helper) { if (maybe_path.empty() || maybe_path.back() != '/') maybe_path += '/'; - arg1->value = maybe_path + "{shard}"; - arg2->value = maybe_replica + "{replica}"; + arg1->value = maybe_path + "auto_{shard}"; + arg2->value = maybe_replica + "auto_{replica}"; return; } - LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); - - if (maybe_shard_macros && maybe_replica_macros) - return; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. " "If you really want to specify it explicitly, then you should use some macros " diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 56e311a5e11..c4d75c65289 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -194,6 +194,9 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file)) if not args.show_db_name: os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file)) + if args.replicated_database: + os.system("LC_ALL=C sed -i -e 's|/auto_{{shard}}||g' {file}".format(file=stdout_file)) + os.system("LC_ALL=C sed -i -e 's|auto_{{replica}}||g' {file}".format(file=stdout_file)) stdout = open(stdout_file, 'rb').read() if os.path.exists(stdout_file) else b'' stdout = str(stdout, errors='replace', encoding='utf-8') @@ -209,8 +212,12 @@ def need_retry(stderr): def get_processlist(args): try: + query = b"SHOW PROCESSLIST FORMAT Vertical" + if args.replicated_database: + query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ + b"FROM clusterAllReplicas('r', system.processes) WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10) + (stdout, _) = clickhouse_proc.communicate(query, timeout=10) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) diff --git a/tests/config/install.sh b/tests/config/install.sh index 76cca9956cb..0fc590b7ab9 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -71,25 +71,25 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 # And we have to copy configs... + mkdir /etc/clickhouse-server1 mkdir /etc/clickhouse-server2 - mkdir /etc/clickhouse-server3 + chown clickhouse /etc/clickhouse-server1 chown clickhouse /etc/clickhouse-server2 - chown clickhouse /etc/clickhouse-server3 + chgrp clickhouse /etc/clickhouse-server1 chgrp clickhouse /etc/clickhouse-server2 - chgrp clickhouse /etc/clickhouse-server3 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server1 sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 - sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server3 + rm /etc/clickhouse-server1/config.d/macros.xml rm /etc/clickhouse-server2/config.d/macros.xml - rm /etc/clickhouse-server3/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server3/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server1/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server2/config.d/macros.xml + sudo mkdir /var/lib/clickhouse1 sudo mkdir /var/lib/clickhouse2 - sudo mkdir /var/lib/clickhouse3 + sudo chown clickhouse /var/lib/clickhouse1 sudo chown clickhouse /var/lib/clickhouse2 - sudo chown clickhouse /var/lib/clickhouse3 + sudo chgrp clickhouse /var/lib/clickhouse1 sudo chgrp clickhouse /var/lib/clickhouse2 - sudo chgrp clickhouse /var/lib/clickhouse3 fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 0fd760d73d5..163ebf1af23 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '1') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '1{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; CREATE TABLE part_header_r2(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '2') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, @@ -39,10 +39,10 @@ SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index bbc2d957937..bf116689ce4 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -15,7 +15,7 @@ CREATE TABLE elog ( engine_id UInt32, referrer String ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog', 'test') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog/{shard}', '{replica}') PARTITION BY date ORDER BY (engine_id) SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;" @@ -28,35 +28,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 3, 'h $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 3 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 4 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 5 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" diff --git a/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql b/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql index 0086ec5c2a3..d8249a603ff 100644 --- a/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql +++ b/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql @@ -8,13 +8,13 @@ CREATE TABLE versioned_collapsing_table( sign Int8, version UInt16 ) -ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table', '1', sign, version) +ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table/{shard}', '{replica}', sign, version) PARTITION BY d ORDER BY (key1, key2); INSERT INTO versioned_collapsing_table VALUES (toDate('2019-10-10'), 1, 1, 'Hello', -1, 1); -SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table' and name = 'metadata'; +SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table/s1' and name = 'metadata'; SELECT COUNT() FROM versioned_collapsing_table; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference index f79be33624b..1f7146dfe48 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference @@ -2,4 +2,4 @@ 10 10 24 -CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions\', \'1\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index 659cc060f32..63a3069c518 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -5,7 +5,7 @@ CREATE TABLE replicated_mutations_empty_partitions key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') ORDER BY key PARTITION by key; @@ -13,7 +13,7 @@ INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number SELECT count(distinct value) FROM replicated_mutations_empty_partitions; -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4'; @@ -21,7 +21,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9'; -- still ten records -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference index cb1eace24a2..0d6ae8338a8 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference @@ -4,13 +4,13 @@ 499999500000 499999500000 Metadata version on replica 1 equal with first replica, OK -CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 2 equal with first replica, OK -CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 3 equal with first replica, OK -CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 4 equal with first replica, OK -CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 5 equal with first replica, OK -CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh index bfa68328c06..b2570cf97f3 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh @@ -11,7 +11,9 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill/{shard}', '{replica}$i') ORDER BY key + SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_kill_1 SELECT number, toString(number) FROM numbers(1000000)" @@ -77,9 +79,9 @@ while true; do done -metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'") +metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'") for i in $(seq $REPLICAS); do - replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'") + replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'") if [ "$metadata_version" != "$replica_metadata_version" ]; then echo "Metadata version on replica $i differs from the first replica, FAIL" else diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index 2fc177c812e..e491dd9e091 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -1,16 +1,14 @@ block_numbers blocks -1 +r1 ======== block_numbers blocks -1 +r1 ======== block_numbers blocks ======== -1 failed_parts last_part -leader_election-0000000000 parallel diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index d4126098c7c..4eb1a73ef6b 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -3,17 +3,19 @@ DROP TABLE IF EXISTS sample_table; CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in', '1') +ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in/{shard}', '{replica}') ORDER BY tuple(); -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in' AND name like 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/replicas' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in') AND name LIKE 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/replicas') ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in','/clickhouse/01700_system_zookeeper_path_in/replicas') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1', + '/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/', name) FROM system.zookeeper WHERE (path = '/clickhouse/01700_system_zookeeper_path_in')) ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/s1/', name) + FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/01700_system_zookeeper_path_in/s1')) ORDER BY name; DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index b1e032e348d..f0fab6b42df 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -117,6 +117,14 @@ "01148_zookeeper_path_macros_unfolding", "01294_system_distributed_on_cluster", "01269_create_with_null", + "01451_replicated_detach_drop_and_quorum", + "01188_attach_table_from_path", + /// user_files + "01721_engine_file_truncate_on_insert", + /// Fails due to additional replicas or shards + "01650_drop_part_and_deduplication_zookeeper", + "01532_execute_merges_on_single_replica", + "01509_parallel_quorum_insert_no_replicas", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From 10696fed4dc9eb7524d04bc2cfc5cdbda8f6ff66 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 18 Mar 2021 17:21:39 +0300 Subject: [PATCH 046/266] Update 00953_zookeeper_suetin_deduplication_bug.sh --- .../0_stateless/00953_zookeeper_suetin_deduplication_bug.sh | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index 98d0f5fad42..baa2b0cf53f 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -15,11 +15,7 @@ CREATE TABLE elog ( engine_id UInt32, referrer String ) -<<<<<<< HEAD -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog/{shard}', '{replica}') -======= -ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog', 'test') ->>>>>>> master +ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/{shard}', '{replica}') PARTITION BY date ORDER BY (engine_id) SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;" From ce31d4608b9a633567a1ec949e84bbacca3a0364 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Mar 2021 22:13:21 +0300 Subject: [PATCH 047/266] fix tests --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- ...01396_inactive_replica_cleanup_nodes_zookeeper.sh | 12 ++++++------ tests/queries/skip_list.json | 3 ++- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index b69e76697b0..63f4d89b216 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -125,7 +125,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr }); if (!processed) - throw Exception(ErrorCodes::UNFINISHED, "Timeout: Cannot enqueue query on this replica," + throw Exception(ErrorCodes::UNFINISHED, "Timeout: Cannot enqueue query on this replica, " "most likely because replica is busy with previous queue entries"); } diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index b604ace85cc..693580bc270 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -12,8 +12,8 @@ SCALE=5000 $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; - CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', '1') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; - CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', '2') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '2{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; DETACH TABLE r2; " @@ -29,16 +29,16 @@ for _ in {1..60}; do done -$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r' AND name = 'log'"; +$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1' AND name = 'log'"; echo -e '\n---\n'; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/1' AND name = 'is_lost'"; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/2' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/1r1' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; echo -e '\n---\n'; $CLICKHOUSE_CLIENT --query "ATTACH TABLE r2" $CLICKHOUSE_CLIENT --receive_timeout 600 --query "SYSTEM SYNC REPLICA r2" # Need to increase timeout, otherwise it timed out in debug build -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/2' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 66f10021359..28d6d2da939 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -119,12 +119,13 @@ "01269_create_with_null", "01451_replicated_detach_drop_and_quorum", "01188_attach_table_from_path", + "01149_zookeeper_mutation_stuck_after_replace_partition", /// user_files "01721_engine_file_truncate_on_insert", /// Fails due to additional replicas or shards + "quorum", "01650_drop_part_and_deduplication_zookeeper", "01532_execute_merges_on_single_replica", - "01509_parallel_quorum_insert_no_replicas", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From 60ef7147bcc17e6715f87b8051556b16e04e8daa Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 19 Mar 2021 07:24:43 +0300 Subject: [PATCH 048/266] trigger CI --- tests/queries/0_stateless/01700_system_zookeeper_path_in.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index 4eb1a73ef6b..02457a956a1 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -8,6 +8,7 @@ ORDER BY tuple(); SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; + SELECT '========'; SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; From f3c0acf753ab7c2a249d3baf08267a200e8a9635 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 20 Mar 2021 13:41:58 +0300 Subject: [PATCH 049/266] Small fix before the second start --- docs/en/sql-reference/table-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index 75e88fdf50d..d65a18ab985 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -29,10 +29,10 @@ You can use table functions in: | [remote](../../sql-reference/table-functions/remote.md) | Allows you to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md)-engine table. | | [url](../../sql-reference/table-functions/url.md) | Creates a [Url](../../engines/table-engines/special/url.md)-engine table. | | [mysql](../../sql-reference/table-functions/mysql.md) | Creates a [MySQL](../../engines/table-engines/integrations/mysql.md)-engine table. | +| [postgresql](../../sql-reference/table-functions/postgresql.md) | Creates a [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)-engine table. | | [jdbc](../../sql-reference/table-functions/jdbc.md) | Creates a [JDBC](../../engines/table-engines/integrations/jdbc.md)-engine table. | | [odbc](../../sql-reference/table-functions/odbc.md) | Creates a [ODBC](../../engines/table-engines/integrations/odbc.md)-engine table. | | [hdfs](../../sql-reference/table-functions/hdfs.md) | Creates a [HDFS](../../engines/table-engines/integrations/hdfs.md)-engine table. | | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | -| [postgresql](../../sql-reference/table-functions/postgresql.md) | Creates a [PostgreSQL](../../engines/table-engines/integrations/postgresql.md)-engine table. | [Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/) From 22444045f0e45b56a21a8a3ddad84d2e250706ac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Mar 2021 16:18:14 +0300 Subject: [PATCH 050/266] CollapsingSortedAlgorithm should not return more then index_granularity rows. --- .../Algorithms/CollapsingSortedAlgorithm.cpp | 33 ++++++++++++++----- .../Algorithms/CollapsingSortedAlgorithm.h | 2 +- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index ccb66259e2e..5b12179513e 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -66,14 +66,16 @@ void CollapsingSortedAlgorithm::insertRow(RowRef & row) merged_data.insertRow(*row.all_columns, row.row_num, row.owned_chunk->getNumRows()); } -void CollapsingSortedAlgorithm::insertRows() +std::optional CollapsingSortedAlgorithm::insertRows() { if (count_positive == 0 && count_negative == 0) { /// No input rows have been read. - return; + return {}; } + std::optional res; + if (last_is_positive || count_positive != count_negative) { if (count_positive <= count_negative && !only_positive_sign) @@ -86,6 +88,9 @@ void CollapsingSortedAlgorithm::insertRows() if (count_positive >= count_negative) { + if (merged_data.hasEnoughRows()) + res = merged_data.pull(); + insertRow(last_positive_row); if (out_row_sources_buf) @@ -107,10 +112,16 @@ void CollapsingSortedAlgorithm::insertRows() out_row_sources_buf->write( reinterpret_cast(current_row_sources.data()), current_row_sources.size() * sizeof(RowSourcePart)); + + return res; } IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() { + /// Rare case, which may happen when index_granularity is 1, but we needed to insert 2 rows inside insertRows(). + if (merged_data.hasEnoughRows()) + return Status(merged_data.pull()); + /// Take rows in required order and put them into `merged_data`, while the rows are no more than `max_block_size` while (queue.isValid()) { @@ -132,15 +143,14 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() setRowRef(last_row, current); bool key_differs = !last_row.hasEqualSortColumnsWith(current_row); - - /// if there are enough rows and the last one is calculated completely - if (key_differs && merged_data.hasEnoughRows()) - return Status(merged_data.pull()); - if (key_differs) { + /// if there are enough rows and the last one is calculated completely + if (merged_data.hasEnoughRows()) + return Status(merged_data.pull()); + /// We write data for the previous primary key. - insertRows(); + auto res = insertRows(); current_row.swap(last_row); @@ -151,6 +161,9 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() first_negative_pos = 0; last_positive_pos = 0; current_row_sources.resize(0); + + if (res) + return Status(std::move(*res)); } /// Initially, skip all rows. On insert, unskip "corner" rows. @@ -194,7 +207,9 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() } } - insertRows(); + if (auto res = insertRows()) + return Status(std::move(*res)); + return Status(merged_data.pull(), true); } diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 028715f715b..e49137c2082 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -66,7 +66,7 @@ private: void reportIncorrectData(); void insertRow(RowRef & row); - void insertRows(); + std::optional insertRows(); }; } From cf83bbd98a8c824e85f1dede80c4d689be8f9303 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Mar 2021 16:51:44 +0300 Subject: [PATCH 051/266] Add crashing test --- ...psing_merge_adaptive_granularity.reference | 2 + ..._collapsing_merge_adaptive_granularity.sql | 53 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference create mode 100644 tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql diff --git a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference new file mode 100644 index 00000000000..30dcbd3b98e --- /dev/null +++ b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference @@ -0,0 +1,2 @@ +-8191 8193 +-8191 8191 diff --git a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql new file mode 100644 index 00000000000..bbc56ec9bb1 --- /dev/null +++ b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql @@ -0,0 +1,53 @@ +DROP TABLE IF EXISTS collapsing_table; +SET optimize_on_insert = 0; + +CREATE TABLE collapsing_table +( + key UInt64, + value UInt64, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY key +SETTINGS + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0; + +INSERT INTO collapsing_table SELECT if(number == 8192, 8191, number), 1, if(number == 8192, +1, -1) FROM numbers(8193); + +SELECT sum(Sign), count() from collapsing_table; + +OPTIMIZE TABLE collapsing_table FINAL; + +SELECT sum(Sign), count() from collapsing_table; + +DROP TABLE IF EXISTS collapsing_table; + + +DROP TABLE IF EXISTS collapsing_suspicious_granularity; + +CREATE TABLE collapsing_suspicious_granularity +( + key UInt64, + value UInt64, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY key +SETTINGS + vertical_merge_algorithm_min_rows_to_activate=0, + vertical_merge_algorithm_min_columns_to_activate=0, + min_bytes_for_wide_part = 0 + index_granularity = 1; + +INSERT INTO collapsing_suspicious_granularity VALUES (1, 1, -1) (1, 1, 1); + +SELECT sum(Sign), count() from collapsing_suspicious_granularity; + +OPTIMIZE TABLE collapsing_suspicious_granularity FINAL; + +SELECT sum(Sign), count() from collapsing_suspicious_granularity; + + +DROP TABLE IF EXISTS collapsing_suspicious_granularity; From 5789507e8b10b6d4e5a322384f3d78ed7f5f7297 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 17 Mar 2021 19:22:56 +0300 Subject: [PATCH 052/266] Investigating, why the checksums may match when they shouldn't. --- .../ReplicatedMergeTreeBlockOutputStream.cpp | 7 +++++ src/Storages/StorageReplicatedMergeTree.cpp | 18 ++++++++---- .../helpers/corrupt_part_data_on_disk.py | 4 +++ .../test_attach_without_fetching/test.py | 28 ++++++++++++++----- 4 files changed, 44 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index c91fa3a86d0..317702b0a80 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -270,6 +270,13 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// the MutableDataPartPtr here, we already have the data thus being able to /// calculate the checksums. log_entry.part_checksum = part->checksums.getTotalChecksumHex(); + + for (auto && [name, checksum] : part->checksums.files) + LOG_TRACE(log, "> File {}, file size {}, file hash ({}, {})", name, checksum.file_size, + checksum.file_hash.first, checksum.file_hash.second); + + LOG_TRACE(log, "Checksums files: {}, clean checksum: {}", part->checksums.files.size(), + log_entry.part_checksum); } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 46920258567..e80d56f080e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1360,7 +1360,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const MergeTreePartInfo actual_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); const String part_new_name = actual_part_info.getPartName(); - LOG_TRACE(log, "Trying to attach part {} from local data", part_new_name); + LOG_TRACE(log, "Trying to attach part {}, checksum {}", part_new_name, entry.part_checksum); for (const DiskPtr & disk : getStoragePolicy()->getDisks()) for (const auto it = disk->iterateDirectory(relative_data_path + "detached/"); it->isValid(); it->next()) @@ -1381,10 +1381,17 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo // The faster way is to load invalid data and just check the checksums -- they won't match. part->loadColumnsChecksumsIndexes(true, false); + for (auto && [name, checksum] : part->checksums.files) + LOG_TRACE(log, "> File {}, file size {}, file hash ({}, {})", name, checksum.file_size, + checksum.file_hash.first, checksum.file_hash.second); + + LOG_TRACE(log, "Checksums files: {}, path: {}, part checksum {}", + part->checksums.files.size(), part->getFullPath(), part->checksums.getTotalChecksumHex()); + if (entry.part_checksum == part->checksums.getTotalChecksumHex()) { - part->is_temp = true; - part->modification_time = disk->getLastModified(part_path).epochTime(); + //part->loadColumnsChecksumsIndexes(true, true); //not sure if it's needed TODO + //part->modification_time = disk->getLastModified(part->getFullRelativePath()).epochTime(); return part; } } @@ -1433,12 +1440,11 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) { - LOG_TRACE(log, "Found valid part {} to attach from local data, preparing the transaction", - part->name); + LOG_TRACE(log, "Found valid part to attach from local data, preparing the transaction"); Transaction transaction(*this); - renameTempPartAndReplace(part, nullptr, &transaction); + renameTempPartAndAdd(part, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); writePartLog(PartLogElement::Type::NEW_PART, {}, diff --git a/tests/integration/helpers/corrupt_part_data_on_disk.py b/tests/integration/helpers/corrupt_part_data_on_disk.py index 63bd2531063..1a6f384da9e 100644 --- a/tests/integration/helpers/corrupt_part_data_on_disk.py +++ b/tests/integration/helpers/corrupt_part_data_on_disk.py @@ -5,6 +5,10 @@ def corrupt_part_data_on_disk(node, table, part_name): corrupt_part_data_by_path(node, part_path) def corrupt_part_data_by_path(node, part_path): + print("Corrupting part", part_path, "at", node.name) + print("Will corrupt: ", + node.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin | head -n 1'.format(p=part_path)])) + node.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin | head -n 1 | xargs -I{{}} sh -c \'echo "1" >> $1\' -- {{}}'.format( p=part_path)], privileged=True) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 3ab6fc3e525..362c17aeef4 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -89,20 +89,34 @@ def test_attach_without_fetching(start_cluster): # Replica 3 should download the data from replica 1 as there is no local data. # Replica 2 should also download the data from 1 as the checksums won't match. print("Checking attach with corrupted checksums") - corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") + + # corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") + + print("Before deleting:", node_2.exec_in_container(['bash', '-c', + 'cd {p} && ls *.bin'.format( + p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True)) + + node_2.exec_in_container(['bash', '-c', + 'cd {p} && rm -fr *.bin'.format( + p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True) + + print("After deleting:", node_2.exec_in_container(['bash', '-c', + 'cd {p} && ls'.format( + p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True)) + node_1.query("ALTER TABLE test ATTACH PARTITION 1") check_data([node_1, node_2, node_3], detached_parts=[0]) # 4. Attach the first part and check if it has been fetched correctly. - # Replica 1 should attach the local data from detached/. - # Replica 3 should download the data from replica 1 as there is no local data and other connections are broken. + # Replica 2 should attach the local data from detached/. + # Replica 3 should download the data from replica 2 as there is no local data and other connections are broken. print("Checking attach with valid checksums") with PartitionManager() as pm: - # If something goes wrong and replica 1 wants to fetch data, the test will fail. - pm.partition_instances(node_1, node_2) - pm.partition_instances(node_3, node_2) + # If something goes wrong and replica 2 wants to fetch data, the test will fail. + pm.partition_instances(node_2, node_1) + pm.partition_instances(node_1, node_3) - node_2.query("ALTER TABLE test ATTACH PART '0_0_0_0'") + node_1.query("ALTER TABLE test ATTACH PART '0_0_0_0'") check_data([node_1, node_2, node_3], detached_parts=[]) From c55a73b75268b13427dce6bab9049dd9f29cd0ce Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 22 Mar 2021 16:27:35 +0300 Subject: [PATCH 053/266] Added the solution to handle the corruption case When the part data (e.g. data.bin) is corrupted, but the checksums.txt is present -- explicitly deleting the checksums.txt. Removed the extra logging, changes some exceptions message. --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 +++- .../ReplicatedMergeTreeBlockOutputStream.cpp | 7 ---- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 6 ++- src/Storages/StorageReplicatedMergeTree.cpp | 40 +++++++------------ .../test_attach_without_fetching/test.py | 4 -- 5 files changed, 24 insertions(+), 40 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1f18c894465..0fc8b4dba9d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -754,7 +754,8 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() void IMergeTreeDataPart::loadChecksums(bool require) { - String path = getFullRelativePath() + "checksums.txt"; + const String path = getFullRelativePath() + "checksums.txt"; + if (volume->getDisk()->exists(path)) { auto buf = openForReading(volume->getDisk(), path); @@ -769,12 +770,14 @@ void IMergeTreeDataPart::loadChecksums(bool require) else { if (require) - throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); + throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No checksums.txt in part {}", name); /// If the checksums file is not present, calculate the checksums and write them to disk. /// Check the data while we are at it. LOG_WARNING(storage.log, "Checksums for part {} not found. Will calculate them from data on disk.", name); + checksums = checkDataPart(shared_from_this(), false); + { auto out = volume->getDisk()->writeFile(getFullRelativePath() + "checksums.txt.tmp", 4096); checksums.write(*out); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 317702b0a80..c91fa3a86d0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -270,13 +270,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( /// the MutableDataPartPtr here, we already have the data thus being able to /// calculate the checksums. log_entry.part_checksum = part->checksums.getTotalChecksumHex(); - - for (auto && [name, checksum] : part->checksums.files) - LOG_TRACE(log, "> File {}, file size {}, file hash ({}, {})", name, checksum.file_size, - checksum.file_hash.first, checksum.file_hash.second); - - LOG_TRACE(log, "Checksums files: {}, clean checksum: {}", part->checksums.files.size(), - log_entry.part_checksum); } else log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index c1e425d3a06..a62ab5cd56d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -1,4 +1,5 @@ #include +#include "Access/IAccessEntity.h" #include #include @@ -141,7 +142,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const break; default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown log entry type: {}", static_cast(type)); } out << '\n'; @@ -161,7 +162,8 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> "format version: " >> format_version >> "\n"; if (format_version < 1 || format_version >= FORMAT_LAST) - throw Exception("Unknown ReplicatedMergeTreeLogEntry format version: " + DB::toString(format_version), ErrorCodes::UNKNOWN_FORMAT_VERSION); + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown ReplicatedMergeTreeLogEntry format version: {}", + DB::toString(format_version)); if (format_version >= FORMAT_WITH_CREATE_TIME) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e80d56f080e..e7acd1f91ae 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1377,21 +1377,17 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, part_info, volume, part_path); - // We don't check consistency as in that case this method will throw. - // The faster way is to load invalid data and just check the checksums -- they won't match. - part->loadColumnsChecksumsIndexes(true, false); - - for (auto && [name, checksum] : part->checksums.files) - LOG_TRACE(log, "> File {}, file size {}, file hash ({}, {})", name, checksum.file_size, - checksum.file_hash.first, checksum.file_hash.second); - - LOG_TRACE(log, "Checksums files: {}, path: {}, part checksum {}", - part->checksums.files.size(), part->getFullPath(), part->checksums.getTotalChecksumHex()); + /// We don't check consistency as in that case this method may throw. + /// The faster way is to load invalid data and just check the checksums -- they won't match. + /// The issue here is that one of data part files may be corrupted (e.g. data.bin), but the + /// pre-calculated checksums.txt is correct, so that could lead to the invalid part being attached. + /// So we explicitly remove it and force recalculation. + disk->removeFileIfExists(part->getFullRelativePath() + "checksums.txt"); + part->loadColumnsChecksumsIndexes(false, false); if (entry.part_checksum == part->checksums.getTotalChecksumHex()) { - //part->loadColumnsChecksumsIndexes(true, true); //not sure if it's needed TODO - //part->modification_time = disk->getLastModified(part->getFullRelativePath()).epochTime(); + part->modification_time = disk->getLastModified(part->getFullRelativePath()).epochTime(); return part; } } @@ -1447,11 +1443,8 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) renameTempPartAndAdd(part, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); - writePartLog(PartLogElement::Type::NEW_PART, {}, - 0, // well, not really, but don't have the idea how to measure the time here TODO - part->name, part, - {part}, // not sure whether the initial parts vector should be empty or contain the part itself TODO - nullptr); + writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, + part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr); return true; } @@ -1492,8 +1485,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) case LogEntry::ALTER_METADATA: return executeMetadataAlter(entry); default: - throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type)), - ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected log entry type: {}", static_cast(entry.type)); } if (do_fetch) @@ -1851,18 +1843,16 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) - throw Exception("Too many total fetches from replicas, maximum: " + - storage_settings_ptr->replicated_max_parallel_fetches.toString(), - ErrorCodes::TOO_MANY_FETCHES); + throw Exception(ErrorCodes::TOO_MANY_FETCHES, "Too many total fetches from replicas, maximum: {} ", + storage_settings_ptr->replicated_max_parallel_fetches.toString()); ++total_fetches; SCOPE_EXIT({--total_fetches;}); if (storage_settings_ptr->replicated_max_parallel_fetches_for_table && current_table_fetches >= storage_settings_ptr->replicated_max_parallel_fetches_for_table) - throw Exception("Too many fetches from replicas for table, maximum: " + - storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString(), - ErrorCodes::TOO_MANY_FETCHES); + throw Exception(ErrorCodes::TOO_MANY_FETCHES, "Too many fetches from replicas for table, maximum: {}", + storage_settings_ptr->replicated_max_parallel_fetches_for_table.toString()); ++current_table_fetches; SCOPE_EXIT({--current_table_fetches;}); diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 362c17aeef4..67e2f44c671 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -100,10 +100,6 @@ def test_attach_without_fetching(start_cluster): 'cd {p} && rm -fr *.bin'.format( p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True) - print("After deleting:", node_2.exec_in_container(['bash', '-c', - 'cd {p} && ls'.format( - p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True)) - node_1.query("ALTER TABLE test ATTACH PARTITION 1") check_data([node_1, node_2, node_3], detached_parts=[0]) From d03b1312eb2e559d996973163cfa994881118f7a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 Mar 2021 17:31:43 +0300 Subject: [PATCH 054/266] update skip_list --- tests/queries/skip_list.json | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 28d6d2da939..5d11dc871d6 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -126,6 +126,8 @@ "quorum", "01650_drop_part_and_deduplication_zookeeper", "01532_execute_merges_on_single_replica", + "00652_replicated_mutations_default_database_zookeeper", + "00620_optimize_on_nonleader_replica_zookeeper", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From 35ff8925dfba25a9506118a5dcf03ca26b316528 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Mar 2021 17:43:45 +0300 Subject: [PATCH 055/266] Fix crash. --- .../Merges/Algorithms/CollapsingSortedAlgorithm.cpp | 4 ++++ .../01764_collapsing_merge_adaptive_granularity.reference | 4 +++- .../01764_collapsing_merge_adaptive_granularity.sql | 2 +- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 5b12179513e..02ee6e94d40 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -208,7 +208,11 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() } if (auto res = insertRows()) + { + count_positive = 0; + count_negative = 0; return Status(std::move(*res)); + } return Status(merged_data.pull(), true); } diff --git a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference index 30dcbd3b98e..0f128a62bbb 100644 --- a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference +++ b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.reference @@ -1,2 +1,4 @@ -8191 8193 --8191 8191 +-8191 8193 +0 2 +0 2 diff --git a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql index bbc56ec9bb1..ca6465154ea 100644 --- a/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql +++ b/tests/queries/0_stateless/01764_collapsing_merge_adaptive_granularity.sql @@ -38,7 +38,7 @@ ORDER BY key SETTINGS vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0, - min_bytes_for_wide_part = 0 + min_bytes_for_wide_part = 0, index_granularity = 1; INSERT INTO collapsing_suspicious_granularity VALUES (1, 1, -1) (1, 1, 1); From da67e06aa0756302da4a50b321e4acd663adada8 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 22 Mar 2021 17:52:21 +0300 Subject: [PATCH 056/266] Added another test case to handle missing part data --- src/Storages/StorageReplicatedMergeTree.cpp | 16 +++++++++-- .../test_attach_without_fetching/test.py | 28 +++++++++++++------ 2 files changed, 34 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e7acd1f91ae..a9fa196d4a2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1377,13 +1377,25 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, part_info, volume, part_path); - /// We don't check consistency as in that case this method may throw. + /// We don't check consistency as in that case this method may throw if the data is corrupted. /// The faster way is to load invalid data and just check the checksums -- they won't match. /// The issue here is that one of data part files may be corrupted (e.g. data.bin), but the /// pre-calculated checksums.txt is correct, so that could lead to the invalid part being attached. /// So we explicitly remove it and force recalculation. disk->removeFileIfExists(part->getFullRelativePath() + "checksums.txt"); - part->loadColumnsChecksumsIndexes(false, false); + + /// The try-catch here is for the case when one of the part's mandatory data files is missing, so the + /// internal loading method in the IMergeTreeDataPart throws. + /// We can't rethrow upper as in that case the fetching will not happen. + /// This situation is not thought to happen often, so handling the exception is ok. + try + { + part->loadColumnsChecksumsIndexes(false, false); + } + catch (const Exception&) // we don't really want to know what exactly happened + { + continue; + } if (entry.part_checksum == part->checksums.getTotalChecksumHex()) { diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 67e2f44c671..a79a7babc08 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -72,15 +72,19 @@ def test_attach_without_fetching(start_cluster): check_data([node_1, node_2], detached_parts=[]) - # 1. Detach the first two partitions/parts on the replicas + # 1. # This part will be fetched from other replicas as it would be missing in the detached/ folder and # also attached locally. node_1.query("ALTER TABLE test DETACH PART '0_0_0_0'") # This partition will be just fetched from other replicas as the checksums won't match # (we'll manually break the data). node_1.query("ALTER TABLE test DETACH PARTITION 1") + # This partition will be just fetched from other replicas as the part data will be corrupted with one of the + # files missing. + node_1.query("ALTER TABLE test DETACH PARTITION 2") - check_data([node_1, node_2], detached_parts=[0, 1]) + + check_data([node_1, node_2], detached_parts=[0, 1, 2]) # 2. Create the third replica fill_node(node_3) @@ -88,22 +92,30 @@ def test_attach_without_fetching(start_cluster): # 3. Break the part data on the second node to corrupt the checksums. # Replica 3 should download the data from replica 1 as there is no local data. # Replica 2 should also download the data from 1 as the checksums won't match. - print("Checking attach with corrupted checksums") - - # corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") + print("Checking attach with corrupted part data with files missing") print("Before deleting:", node_2.exec_in_container(['bash', '-c', 'cd {p} && ls *.bin'.format( - p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True)) + p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True)) node_2.exec_in_container(['bash', '-c', 'cd {p} && rm -fr *.bin'.format( - p="/var/lib/clickhouse/data/default/test/detached/1_0_0_0")], privileged=True) + p="/var/lib/clickhouse/data/default/test/detached/2_0_0_0")], privileged=True) + + node_1.query("ALTER TABLE test ATTACH PARTITION 2") + check_data([node_1, node_2, node_3], detached_parts=[0, 1]) + + # 4. Break the part data on the second node to corrupt the checksums. + # Replica 3 should download the data from replica 1 as there is no local data. + # Replica 2 should also download the data from 1 as the checksums won't match. + print("Checking attach with corrupted part data with all of the files present") + + corrupt_part_data_by_path(node_2, "/var/lib/clickhouse/data/default/test/detached/1_0_0_0") node_1.query("ALTER TABLE test ATTACH PARTITION 1") check_data([node_1, node_2, node_3], detached_parts=[0]) - # 4. Attach the first part and check if it has been fetched correctly. + # 5. Attach the first part and check if it has been fetched correctly. # Replica 2 should attach the local data from detached/. # Replica 3 should download the data from replica 2 as there is no local data and other connections are broken. print("Checking attach with valid checksums") From 761c2321b621a144ec7aa69607f40b5b227dcda1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Mar 2021 19:48:03 +0300 Subject: [PATCH 057/266] Update test. --- ...6_adaptive_index_granularity_collapsing_merge_tree.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference index e5e283f754b..3a176a17f5a 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference @@ -1,7 +1,7 @@ 4 4 8 -7 +8 ----- 4 1 From 166054291e04714c2125d4ac311107c45a0aff5f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Mar 2021 10:37:22 +0300 Subject: [PATCH 058/266] Less flaky test --- .../00926_adaptive_index_granularity_collapsing_merge_tree.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql index d4c19cbe8f2..58b266f106f 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.sql @@ -58,7 +58,7 @@ OPTIMIZE TABLE four_rows_per_granule FINAL; SELECT COUNT(*) FROM four_rows_per_granule; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1; +SELECT sum(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1); From d7fc5e69f4d4b6e3f18d0de1b9c3327f8f895653 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Mar 2021 11:18:49 +0300 Subject: [PATCH 059/266] Update CollapsingSortedAlgorithm.h Added comment. --- src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index e49137c2082..18ebaad5596 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -66,6 +66,10 @@ private: void reportIncorrectData(); void insertRow(RowRef & row); + + /// Insert ready rows into merged_data. We may want to insert 0, 1 or 2 rows. + /// It may happen that 2 rows is going to be inserted and, but merged data has free space only for 1 row. + /// In this case, Chunk with ready is pulled from merged_data before the second insertion. std::optional insertRows(); }; From ed19864e5be5b51208e764a8e3c35a3d0262aa02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Mar 2021 11:25:45 +0300 Subject: [PATCH 060/266] Update CollapsingSortedAlgorithm.cpp Add comment. --- .../Merges/Algorithms/CollapsingSortedAlgorithm.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 02ee6e94d40..0db99fc7b0e 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -162,6 +162,9 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() last_positive_pos = 0; current_row_sources.resize(0); + /// Here we can return ready chunk. + /// Next iteration, last_row == current_row, and all the counters are zeroed. + /// So, current_row should be correctly processed. if (res) return Status(std::move(*res)); } @@ -209,6 +212,8 @@ IMergingAlgorithm::Status CollapsingSortedAlgorithm::merge() if (auto res = insertRows()) { + /// Queue is empty, and we have inserted all the rows. + /// Set counter to zero so that insertRows() will return immediately next time. count_positive = 0; count_negative = 0; return Status(std::move(*res)); From ea6b8486e92478a7aadc1ef3ca4a31a77e2120f4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Feb 2021 14:57:16 +0300 Subject: [PATCH 061/266] Try to fix issue #20497 --- src/Interpreters/PredicateRewriteVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index 9e6d5543f2f..e1d80c1954a 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -106,7 +106,7 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con for (const auto & identifier : identifiers) { - const auto & column_name = identifier->shortName(); + const auto & column_name = identifier->name(); const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); /// For lambda functions, we can't always find them in the list of columns From e378c0bf8a805e904e6f8f931cc311ad79c12c4b Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 19 Mar 2021 16:20:19 +0300 Subject: [PATCH 062/266] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 149 ++++++++++++++++-- src/Disks/S3/DiskS3.h | 23 ++- .../config.d/storage_conf_not_restorable.xml | 35 ++++ .../test_merge_tree_s3_restore/test.py | 58 ++++++- 4 files changed, 250 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bb9966eb6ff..bb15df9b9e3 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -648,7 +648,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path) if (send_metadata) { auto revision = ++revision_counter; - const DiskS3::ObjectMetadata object_metadata { + const ObjectMetadata object_metadata { {"from_path", from_path}, {"to_path", to_path} }; @@ -942,7 +942,16 @@ void DiskS3::startup() LOG_INFO(&Poco::Logger::get("DiskS3"), "Starting up disk {}", name); - /// Find last revision. + if (readSchemaVersion(bucket, s3_root_path) < RESTORABLE_SCHEMA_VERSION) + migrateToRestorableSchema(); + + findLastRevision(); + + LOG_INFO(&Poco::Logger::get("DiskS3"), "Disk {} started up", name); +} + +void DiskS3::findLastRevision() +{ UInt64 l = 0, r = LATEST_REVISION; while (l < r) { @@ -954,8 +963,8 @@ void DiskS3::startup() LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check object with revision {}", revision); /// Check file or operation with such revision exists. - if (checkObjectExists(s3_root_path + "r" + revision_str) - || checkObjectExists(s3_root_path + "operations/r" + revision_str)) + if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) + || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; else r = revision - 1; @@ -964,10 +973,124 @@ void DiskS3::startup() LOG_INFO(&Poco::Logger::get("DiskS3"), "Found last revision number {} for disk {}", revision_counter, name); } -bool DiskS3::checkObjectExists(const String & prefix) +int DiskS3::readSchemaVersion(const String & source_bucket, const String & source_path) +{ + int version = 0; + if (!checkObjectExists(source_bucket, source_path + SCHEMA_VERSION_OBJECT)) + return version; + + ReadBufferFromS3 buffer (client, source_bucket, source_path + SCHEMA_VERSION_OBJECT); + readIntText(version, buffer); + + return version; +} + +void DiskS3::saveSchemaVersion(const int & version) +{ + WriteBufferFromS3 buffer (client, bucket, s3_root_path + SCHEMA_VERSION_OBJECT, min_upload_part_size, max_single_part_upload_size); + writeIntText(version, buffer); + buffer.finalize(); +} + +void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & metadata) +{ + Aws::S3::Model::CopyObjectRequest request; + request.SetCopySource(bucket + "/" + key); + request.SetBucket(bucket); + request.SetKey(key); + request.SetMetadata(metadata); + + auto outcome = client->CopyObject(request); + throwIfError(outcome); +} + +void DiskS3::migrateFileToRestorableSchema(const String & path) +{ + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate file {} to restorable schema", metadata_path + path); + + auto meta = readMeta(path); + + for (const auto & [key, _] : meta.s3_objects) + { + ObjectMetadata metadata { + {"path", path} + }; + updateObjectMetadata(s3_root_path + key, metadata); + } +} + +void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & results) +{ + checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. + + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate directory {} to restorable schema", metadata_path + path); + + bool dir_contains_only_files = true; + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (isDirectory(it->path())) + { + dir_contains_only_files = false; + break; + } + + /// The whole directory can be migrated asynchronously. + if (dir_contains_only_files) + { + auto result = getExecutor().execute([this, path] + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + migrateFileToRestorableSchema(it->path()); + }); + + results.push_back(std::move(result)); + } + else + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (!isDirectory(it->path())) + { + auto source_path = it->path(); + auto result = getExecutor().execute([this, source_path] + { + migrateFileToRestorableSchema(source_path); + }); + + results.push_back(std::move(result)); + } + else + migrateToRestorableSchemaRecursive(it->path(), results); + } +} + +void DiskS3::migrateToRestorableSchema() +{ + try + { + LOG_INFO(&Poco::Logger::get("DiskS3"), "Start migration to restorable schema for disk {}", name); + + Futures results; + + migrateToRestorableSchemaRecursive("data/", results); + + for (auto & result : results) + result.wait(); + for (auto & result : results) + result.get(); + + saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); + } + catch (const Exception & e) + { + LOG_ERROR(&Poco::Logger::get("DiskS3"), "Failed to migrate to restorable schema. Code: {}, e.displayText() = {}, Stack trace:\n\n{}", e.code(), e.displayText(), e.getStackTraceString()); + + throw; + } +} + +bool DiskS3::checkObjectExists(const String & source_bucket, const String & prefix) { Aws::S3::Model::ListObjectsV2Request request; - request.SetBucket(bucket); + request.SetBucket(source_bucket); request.SetPrefix(prefix); request.SetMaxKeys(1); @@ -1048,7 +1171,7 @@ struct DiskS3::RestoreInformation void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_information) { - ReadBufferFromFile buffer(metadata_path + restore_file_name, 512); + ReadBufferFromFile buffer(metadata_path + RESTORE_FILE_NAME, 512); buffer.next(); /// Empty file - just restore all metadata. @@ -1083,7 +1206,7 @@ void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_informa void DiskS3::restore() { - if (!exists(restore_file_name)) + if (!exists(RESTORE_FILE_NAME)) return; try @@ -1110,15 +1233,21 @@ void DiskS3::restore() throw Exception("Restoring to the same bucket is allowed only if source path is not a sub-path of configured path in S3 disk", ErrorCodes::BAD_ARGUMENTS); } - ///TODO: Cleanup FS and bucket if previous restore was failed. + if (readSchemaVersion(information.source_bucket, information.source_path) < RESTORABLE_SCHEMA_VERSION) + throw Exception("Source bucket doesn't have restorable schema.", ErrorCodes::BAD_ARGUMENTS); LOG_INFO(&Poco::Logger::get("DiskS3"), "Starting to restore disk {}. Revision: {}, Source bucket: {}, Source path: {}", name, information.revision, information.source_bucket, information.source_path); + LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); + + bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; + removeSharedRecursive("data/", !cleanup_s3); + restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); - Poco::File restore_file(metadata_path + restore_file_name); + Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); restore_file.remove(); LOG_INFO(&Poco::Logger::get("DiskS3"), "Restore disk {} finished", name); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5d9effa16fa..4a5f93866bd 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -25,6 +25,7 @@ class DiskS3 : public IDisk { public: using ObjectMetadata = std::map; + using Futures = std::vector>; friend class DiskS3Reservation; @@ -148,7 +149,16 @@ private: void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata); static String revisionToString(UInt64 revision); - bool checkObjectExists(const String & prefix); + bool checkObjectExists(const String & source_bucket, const String & prefix); + void findLastRevision(); + + int readSchemaVersion(const String & source_bucket, const String & source_path); + void saveSchemaVersion(const int & version); + void updateObjectMetadata(const String & key, const ObjectMetadata & metadata); + void migrateFileToRestorableSchema(const String & path); + void migrateToRestorableSchemaRecursive(const String & path, Futures & results); + void migrateToRestorableSchema(); + Aws::S3::Model::HeadObjectResult headObject(const String & source_bucket, const String & key); void listObjects(const String & source_bucket, const String & source_path, std::function callback); void copyObject(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key); @@ -168,7 +178,7 @@ private: std::shared_ptr proxy_configuration; const String bucket; const String s3_root_path; - const String metadata_path; + String metadata_path; size_t min_upload_part_size; size_t max_single_part_upload_size; size_t min_bytes_for_seek; @@ -179,16 +189,21 @@ private: std::mutex reservation_mutex; std::atomic revision_counter; - static constexpr UInt64 LATEST_REVISION = (static_cast(1)) << 63; + static constexpr UInt64 LATEST_REVISION = std::numeric_limits::max(); static constexpr UInt64 UNKNOWN_REVISION = 0; /// File at path {metadata_path}/restore contains metadata restore information - const String restore_file_name = "restore"; + inline static const String RESTORE_FILE_NAME = "restore"; /// The number of keys listed in one request (1000 is max value) int list_object_keys_size; /// Key has format: ../../r{revision}-{operation} const re2::RE2 key_regexp {".*/r(\\d+)-(\\w+).*"}; + + /// Object contains information about schema version. + inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION"; + /// Version with possibility to backup-restore metadata. + static constexpr int RESTORABLE_SCHEMA_VERSION = 1; }; } diff --git a/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml b/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml new file mode 100644 index 00000000000..c682ddae785 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml @@ -0,0 +1,35 @@ + + + + + s3 + http://minio1:9001/root/another_data/ + minio + minio123 + false + 1 + 0 + + + local + / + + + + + +
+ s3 +
+ + hdd + +
+
+
+
+ + + 0 + +
diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 346d9aced3f..0d08e2686b9 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -1,3 +1,4 @@ +import os import logging import random import string @@ -10,6 +11,20 @@ logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node_not_restorable/configs/config.d/storage_conf_not_restorable.xml') + + +def replace_config(old, new): + config = open(CONFIG_PATH, 'r') + config_lines = config.readlines() + config.close() + config_lines = [line.replace(old, new) for line in config_lines] + config = open(CONFIG_PATH, 'w') + config.writelines(config_lines) + config.close() + + @pytest.fixture(scope="module") def cluster(): try: @@ -26,6 +41,10 @@ def cluster(): "configs/config.d/storage_conf_another_bucket_path.xml", "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml"], user_configs=[], stay_alive=True) + cluster.add_instance("node_not_restorable", main_configs=[ + "configs/config.d/storage_conf_not_restorable.xml", + "configs/config.d/bg_processing_pool_conf.xml", + "configs/config.d/log_conf.xml"], user_configs=[], stay_alive=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -103,7 +122,7 @@ def get_revision_counter(node, backup_number): def drop_table(cluster): yield - node_names = ["node", "node_another_bucket", "node_another_bucket_path"] + node_names = ["node", "node_another_bucket", "node_another_bucket_path", "node_not_restorable"] for node_name in node_names: node = cluster.instances[node_name] @@ -311,3 +330,40 @@ def test_restore_mutations(cluster): assert node_another_bucket.query("SELECT sum(id) FROM s3.test FORMAT Values") == "({})".format(0) assert node_another_bucket.query("SELECT sum(counter) FROM s3.test FORMAT Values") == "({})".format(4096 * 2) assert node_another_bucket.query("SELECT sum(counter) FROM s3.test WHERE id > 0 FORMAT Values") == "({})".format(4096) + + +def test_migrate_to_restorable_schema(cluster): + node = cluster.instances["node_not_restorable"] + + create_table(node, "test") + + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-03', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-04', 4096, -1))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-05', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-05', 4096, -1))) + + replace_config("false", "true") + + node.restart_clickhouse() + + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-06', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-06', 4096, -1))) + + node.query("ALTER TABLE s3.test FREEZE") + revision = get_revision_counter(node, 1) + + assert revision != 0 + + node_another_bucket = cluster.instances["node_another_bucket"] + + create_table(node_another_bucket, "test") + + # Restore to revision before mutation. + node_another_bucket.stop_clickhouse() + drop_s3_metadata(node_another_bucket) + purge_s3(cluster, cluster.minio_bucket_2) + create_restore_file(node_another_bucket, revision=revision, bucket="root", path="another_data") + node_another_bucket.start_clickhouse(10) + + assert node_another_bucket.query("SELECT count(*) FROM s3.test FORMAT Values") == "({})".format(4096 * 6) + assert node_another_bucket.query("SELECT sum(id) FROM s3.test FORMAT Values") == "({})".format(0) From 7e462a77a7f4cf1eb4a3dcd9377287dcc510fa39 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:05:43 +0300 Subject: [PATCH 063/266] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 21 ++++++++++++++++--- .../test_merge_tree_s3_restore/test.py | 2 ++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bb15df9b9e3..6adae24d15d 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -966,6 +966,8 @@ void DiskS3::findLastRevision() if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; + else if (revision == 0) + r = 0; else r = revision - 1; } @@ -999,6 +1001,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met request.SetBucket(bucket); request.SetKey(key); request.SetMetadata(metadata); + request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE); auto outcome = client->CopyObject(request); throwIfError(outcome); @@ -1070,7 +1073,10 @@ void DiskS3::migrateToRestorableSchema() Futures results; - migrateToRestorableSchemaRecursive("data/", results); + if (exists("data")) + migrateToRestorableSchemaRecursive("data/", results); + if (exists("store")) + migrateToRestorableSchemaRecursive("store/", results); for (auto & result : results) result.wait(); @@ -1242,7 +1248,10 @@ void DiskS3::restore() LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; - removeSharedRecursive("data/", !cleanup_s3); + if (exists("data")) + removeSharedRecursive("data/", !cleanup_s3); + if (exists("store")) + removeSharedRecursive("data/", !cleanup_s3); restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); @@ -1250,6 +1259,8 @@ void DiskS3::restore() Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); restore_file.remove(); + saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); + LOG_INFO(&Poco::Logger::get("DiskS3"), "Restore disk {} finished", name); } catch (const Exception & e) @@ -1315,7 +1326,11 @@ void DiskS3::processRestoreFiles(const String & source_bucket, const String & so /// Restore file if object has 'path' in metadata. auto path_entry = object_metadata.find("path"); if (path_entry == object_metadata.end()) - throw Exception("Failed to restore key " + key + " because it doesn't have 'path' in metadata", ErrorCodes::S3_ERROR); + { + /// Such keys can remain after migration, we can skip them. + LOG_WARNING(&Poco::Logger::get("DiskS3"), "Skip key {} because it doesn't have 'path' in metadata", key); + continue; + } const auto & path = path_entry->second; diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 0d08e2686b9..c0ebce68480 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -94,6 +94,8 @@ def create_table(node, table_name, additional_settings=None): def purge_s3(cluster, bucket): minio = cluster.minio_client for obj in list(minio.list_objects(bucket, recursive=True)): + if str(obj.object_name).find(".SCHEMA_VERSION") != -1: + continue minio.remove_object(bucket, obj.object_name) From 9f9419cf144643750463a2113edac46fc51f0a44 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:08:37 +0300 Subject: [PATCH 064/266] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 6adae24d15d..94804985762 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -958,6 +958,9 @@ void DiskS3::findLastRevision() LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check revision in bounds {}-{}", l, r); auto revision = l + (r - l + 1) / 2; + if (revision == 0) + break; + auto revision_str = revisionToString(revision); LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check object with revision {}", revision); @@ -966,8 +969,6 @@ void DiskS3::findLastRevision() if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; - else if (revision == 0) - r = 0; else r = revision - 1; } From 0c0ddefc75c69768d482242f47982226044001f9 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:12:47 +0300 Subject: [PATCH 065/266] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 14 ++++++-------- src/Disks/S3/DiskS3.h | 2 ++ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 94804985762..74c859efb75 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1074,10 +1074,9 @@ void DiskS3::migrateToRestorableSchema() Futures results; - if (exists("data")) - migrateToRestorableSchemaRecursive("data/", results); - if (exists("store")) - migrateToRestorableSchemaRecursive("store/", results); + for (const auto & root : data_roots) + if (exists(root)) + migrateToRestorableSchemaRecursive(root + '/', results); for (auto & result : results) result.wait(); @@ -1249,10 +1248,9 @@ void DiskS3::restore() LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; - if (exists("data")) - removeSharedRecursive("data/", !cleanup_s3); - if (exists("store")) - removeSharedRecursive("data/", !cleanup_s3); + for (const auto & root : data_roots) + if (exists(root)) + removeSharedRecursive(root + '/', !cleanup_s3); restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 4a5f93866bd..5624fd2eee1 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -204,6 +204,8 @@ private: inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION"; /// Version with possibility to backup-restore metadata. static constexpr int RESTORABLE_SCHEMA_VERSION = 1; + /// Directories with data. + const std::vector data_roots {"data", "store"}; }; } From bc1b0aed8523fb23e3ae87a9a81079faa72f27c1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Mar 2021 15:48:29 +0300 Subject: [PATCH 066/266] Correctly handle table alias in PredicateRewriteVisitor --- src/Interpreters/IdentifierSemantic.cpp | 2 +- .../PredicateExpressionsOptimizer.cpp | 6 +++--- .../PredicateExpressionsOptimizer.h | 3 ++- src/Interpreters/PredicateRewriteVisitor.cpp | 19 ++++++++++++------- src/Interpreters/PredicateRewriteVisitor.h | 8 +++++--- .../00597_push_down_predicate.reference | 12 ++++++++++++ .../0_stateless/00597_push_down_predicate.sql | 4 ++++ 7 files changed, 39 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/IdentifierSemantic.cpp b/src/Interpreters/IdentifierSemantic.cpp index a1fc533eb7f..81bd499ea2e 100644 --- a/src/Interpreters/IdentifierSemantic.cpp +++ b/src/Interpreters/IdentifierSemantic.cpp @@ -209,7 +209,7 @@ IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const return canReferColumnToTable(identifier, table_with_columns.table); } -/// Strip qualificators from left side of column name. +/// Strip qualifications from left side of column name. /// Example: 'database.table.name' -> 'name'. void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) { diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index 00b47be408a..476bdaaceea 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -146,7 +146,7 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_e break; /// Skip left and right table optimization is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos], - tables_with_columns[table_pos].columns.getNames()); + tables_with_columns[table_pos]); if (table_element->table_join && isRight(table_element->table_join->as()->kind)) break; /// Skip left table optimization @@ -156,13 +156,13 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_e return is_rewrite_tables; } -bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, Names && table_columns) const +bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const TableWithColumnNamesAndTypes & table_columns) const { if (!table_predicates.empty()) { auto optimize_final = enable_optimize_predicate_expression_to_final_subquery; auto optimize_with = allow_push_predicate_when_subquery_contains_with; - PredicateRewriteVisitor::Data data(context, table_predicates, std::move(table_columns), optimize_final, optimize_with); + PredicateRewriteVisitor::Data data(context, table_predicates, table_columns, optimize_final, optimize_with); PredicateRewriteVisitor(data).visit(table_element); return data.is_rewrite; diff --git a/src/Interpreters/PredicateExpressionsOptimizer.h b/src/Interpreters/PredicateExpressionsOptimizer.h index 8cceda93164..223ac1e8998 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/src/Interpreters/PredicateExpressionsOptimizer.h @@ -33,7 +33,8 @@ private: bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates); - bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, Names && table_columns) const; + bool tryRewritePredicatesToTable( + ASTPtr & table_element, const ASTs & table_predicates, const TableWithColumnNamesAndTypes & table_columns) const; bool tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query); }; diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index e1d80c1954a..d4083e84639 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -17,8 +17,8 @@ namespace DB { PredicateRewriteVisitorData::PredicateRewriteVisitorData( - const Context & context_, const ASTs & predicates_, Names && column_names_, bool optimize_final_, bool optimize_with_) - : context(context_), predicates(predicates_), column_names(column_names_), optimize_final(optimize_final_), optimize_with(optimize_with_) + const Context & context_, const ASTs & predicates_, const TableWithColumnNamesAndTypes & table_columns_, bool optimize_final_, bool optimize_with_) + : context(context_), predicates(predicates_), table_columns(table_columns_), optimize_final(optimize_final_), optimize_with(optimize_with_) { } @@ -42,7 +42,7 @@ void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_q void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &) { - is_rewrite |= rewriteSubquery(select_query, column_names, column_names); + is_rewrite |= rewriteSubquery(select_query, {}); } void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &) @@ -65,7 +65,7 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele const Names & internal_columns = InterpreterSelectQuery( temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames(); - if (rewriteSubquery(*temp_select_query, column_names, internal_columns)) + if (rewriteSubquery(*temp_select_query, internal_columns)) { is_rewrite |= true; select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect())); @@ -89,7 +89,7 @@ static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector identifiers; @@ -106,13 +107,17 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con for (const auto & identifier : identifiers) { + IdentifierSemantic::setColumnShortName(*identifier, table_columns.table); const auto & column_name = identifier->name(); - const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); /// For lambda functions, we can't always find them in the list of columns /// For example: SELECT * FROM system.one WHERE arrayMap(x -> x, [dummy]) = [0] + const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); if (outer_column_iterator != outer_columns.end()) - identifier->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); + { + const Names & column_names = inner_columns.empty() ? outer_columns : inner_columns; + identifier->setShortName(column_names[outer_column_iterator - outer_columns.begin()]); + } } /// We only need to push all the predicates to subquery having diff --git a/src/Interpreters/PredicateRewriteVisitor.h b/src/Interpreters/PredicateRewriteVisitor.h index 02c8b9ca422..1132d93a5ec 100644 --- a/src/Interpreters/PredicateRewriteVisitor.h +++ b/src/Interpreters/PredicateRewriteVisitor.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -24,12 +25,13 @@ public: return true; } - PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, Names && column_names_, bool optimize_final_, bool optimize_with_); + PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, + const TableWithColumnNamesAndTypes & table_columns_, bool optimize_final_, bool optimize_with_); private: const Context & context; const ASTs & predicates; - const Names column_names; + const TableWithColumnNamesAndTypes & table_columns; bool optimize_final; bool optimize_with; @@ -37,7 +39,7 @@ private: void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &); - bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns); + bool rewriteSubquery(ASTSelectQuery & subquery, const Names & inner_columns); }; using PredicateRewriteMatcher = OneTypeMatcher; diff --git a/tests/queries/0_stateless/00597_push_down_predicate.reference b/tests/queries/0_stateless/00597_push_down_predicate.reference index bd1c4791df4..59313c35b81 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -585,3 +585,15 @@ SEMI LEFT JOIN ) AS r USING (id) WHERE r.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 +SELECT value + t1.value AS expr +FROM +( + SELECT + value, + t1.value + FROM test_00597 AS t0 + ALL FULL OUTER JOIN test_00597 AS t1 USING (date) + WHERE (value + `t1.value`) < 3 +) +WHERE expr < 3 +2 diff --git a/tests/queries/0_stateless/00597_push_down_predicate.sql b/tests/queries/0_stateless/00597_push_down_predicate.sql index ec306ac6792..2e3357241ad 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -135,5 +135,9 @@ SELECT * FROM (SELECT * FROM (SELECT * FROM test_00597) AS a ANY LEFT JOIN (SELE EXPLAIN SYNTAX SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1; SELECT * FROM (SELECT * FROM test_00597) ANY INNER JOIN (SELECT * FROM (SELECT * FROM test_00597)) as r USING id WHERE r.id = 1; +-- issue 20497 +EXPLAIN SYNTAX SELECT value + t1.value AS expr FROM (SELECT t0.value, t1.value FROM test_00597 AS t0 FULL JOIN test_00597 AS t1 USING date) WHERE expr < 3; +SELECT value + t1.value AS expr FROM (SELECT t0.value, t1.value FROM test_00597 AS t0 FULL JOIN test_00597 AS t1 USING date) WHERE expr < 3; + DROP TABLE IF EXISTS test_00597; DROP TABLE IF EXISTS test_view_00597; From 9f25ee371918782a277e53507c5fc4add627c92d Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 22 Mar 2021 18:44:44 +0300 Subject: [PATCH 067/266] Replaced removing checksums.txt with version with the explicit data loading and verification. If the function fails, the exception will re-throw upper, cancelling the fetch in the handling of the replicated log entry, but this event will also wake the PartCheckThread, which will issue a re-fetch. --- .../MergeTree/MergeTreeDataPartChecksum.cpp | 5 +---- .../ReplicatedMergeTreePartCheckThread.cpp | 4 +++- src/Storages/StorageReplicatedMergeTree.cpp | 21 +++++++------------ 3 files changed, 12 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 01535ab0dbc..29ff4626a17 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -373,11 +373,8 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar SipHash hash_of_uncompressed_files_state; SipHash uncompressed_hash_of_compressed_files_state; - for (const auto & elem : full_checksums_.files) + for (const auto & [name, checksum] : full_checksums_.files) { - const String & name = elem.first; - const auto & checksum = elem.second; - updateHash(hash_of_all_files_state, name); hash_of_all_files_state.update(checksum.file_hash); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 95119c97470..95883c65abb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -213,7 +213,7 @@ std::pair ReplicatedMergeTreePartCheckThread::findLo /// because our checks of local storage and zookeeper are not consistent. /// If part exists in zookeeper and doesn't exists in local storage definitely require /// to fetch this part. But if we check local storage first and than check zookeeper - /// some background process can successfully commit part between this checks (both to the local stoarge and zookeeper), + /// some background process can successfully commit part between this checks (both to the local storage and zookeeper), /// but checker thread will remove part from zookeeper and queue fetch. bool exists_in_zookeeper = zookeeper->exists(part_path); @@ -234,6 +234,8 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na auto [exists_in_zookeeper, part] = findLocalPart(part_name); + LOG_TRACE(log, "Part {} in zookeeper: {}, locally: {}", part_name, exists_in_zookeeper, part != nullptr); + /// We do not have this or a covering part. if (!part) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a9fa196d4a2..83b44992f6c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1377,23 +1377,14 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, part_info, volume, part_path); - /// We don't check consistency as in that case this method may throw if the data is corrupted. - /// The faster way is to load invalid data and just check the checksums -- they won't match. - /// The issue here is that one of data part files may be corrupted (e.g. data.bin), but the - /// pre-calculated checksums.txt is correct, so that could lead to the invalid part being attached. - /// So we explicitly remove it and force recalculation. - disk->removeFileIfExists(part->getFullRelativePath() + "checksums.txt"); - - /// The try-catch here is for the case when one of the part's mandatory data files is missing, so the - /// internal loading method in the IMergeTreeDataPart throws. - /// We can't rethrow upper as in that case the fetching will not happen. - /// This situation is not thought to happen often, so handling the exception is ok. try { - part->loadColumnsChecksumsIndexes(false, false); + part->loadColumnsChecksumsIndexes(true, true); } - catch (const Exception&) // we don't really want to know what exactly happened + catch (const Exception&) { + /// This method throws if the part data is corrupted or partly missing. In this case, we simply don't + /// process the part. continue; } @@ -3333,12 +3324,16 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); + LOG_TRACE(log, "Candidate replicas: {}", replicas.size()); + for (const String & replica : replicas) { /// We aren't interested in ourself. if (replica == replica_name) continue; + LOG_TRACE(log, "Candidate replica: {}", replica); + if (checkReplicaHavePart(replica, part_name) && (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))) return replica; From 9db74c4dc0194ccad5e6cbe0c7bce1baff5a1ae5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Mar 2021 22:40:24 +0300 Subject: [PATCH 068/266] Add stress test for distributed queries It may founds issue like in [1]: 2021.03.18 19:05:38.783328 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} executeQuery: (from 127.0.0.1:40918, using production parser) select * from dist where key = 0; 2021.03.18 19:05:38.783760 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} StorageDistributed (dist): Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): [1] 2021.03.18 19:05:38.784012 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} ContextAccess (default): Access granted: SELECT(key) ON default.dist 2021.03.18 19:05:38.784410 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} ContextAccess (default): Access granted: SELECT(key) ON default.dist 2021.03.18 19:05:38.784488 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} StorageDistributed (dist): Disabling force_optimize_skip_unused_shards for nested queries (force_optimize_skip_unused_shards_nesting exceeded) 2021.03.18 19:05:38.784572 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} InterpreterSelectQuery: Complete -> Complete 2021.03.18 19:05:38.819063 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} executeQuery: Read 20 rows, 80.00 B in 0.035687783 sec., 560 rows/sec., 2.19 KiB/sec. 2021.03.18 19:05:38.827842 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.03.18 19:05:38.867752 [ 547 ] {} BaseDaemon: ######################################## 2021.03.18 19:05:38.867959 [ 547 ] {} BaseDaemon: (version 21.4.1.1, build id: A0ADEC175BD65E58EA012C47C265E661C32D23B5) (from thread 245) (query_id: 4b1f5ec0-bf2d-478c-a2e1-d312531db206) Received signal Aborted (6) 2021.03.18 19:05:38.868733 [ 547 ] {} BaseDaemon: 2021.03.18 19:05:38.868958 [ 547 ] {} BaseDaemon: Stack trace: 0x7fd1394be18b 0x7fd13949d859 0x10c4c99b 0xd434ee1 0xd434f1a 2021.03.18 19:05:38.870135 [ 547 ] {} BaseDaemon: 3. gsignal @ 0x4618b in /usr/lib/x86_64-linux-gnu/libc-2.31.so 2021.03.18 19:05:38.870383 [ 547 ] {} BaseDaemon: 4. abort @ 0x25859 in /usr/lib/x86_64-linux-gnu/libc-2.31.so 2021.03.18 19:05:38.886783 [ 547 ] {} BaseDaemon: 5. /work3/azat/ch/clickhouse/.cmake/../contrib/libunwind/src/UnwindLevel1.c:396: _Unwind_Resume @ 0x10c4c99b in /usr/bin/clickhouse 2021.03.18 19:05:47.200208 [ 547 ] {} BaseDaemon: 6. ? @ 0xd434ee1 in /usr/bin/clickhouse 2021.03.18 19:05:47.348738 [ 547 ] {} BaseDaemon: 7.1. inlined from /work3/azat/ch/clickhouse/.cmake/../contrib/boost/boost/context/fiber_fcontext.hpp:253: boost::context::fiber::~fiber() 2021.03.18 19:05:47.349118 [ 547 ] {} BaseDaemon: 7.2. inlined from ../contrib/boost/boost/context/fiber_fcontext.hpp:252: boost::context::detail::fiber_record::run(void*) 2021.03.18 19:05:47.349163 [ 547 ] {} BaseDaemon: 7. ../contrib/boost/boost/context/fiber_fcontext.hpp:80: void boost::context::detail::fiber_entry >(boost::context::detail::transfer_t) @ 0xd434f1a in /usr/bin/clickhouse 2021.03.18 19:05:47.618174 [ 547 ] {} BaseDaemon: Calculated checksum of the binary: FF3BA83D0CD648741EEEC242CB1966D9. There is no information about the reference checksum. [1]: https://clickhouse-test-reports.s3.yandex.net/0/1b2ed51ff5e4a3dc45567d4967108f43f680c884/stress_test_(debug).html#fail1 --- .../__init__.py | 0 .../configs/remote_servers.xml | 42 +++++++ .../test_distributed_queries_stress/test.py | 103 ++++++++++++++++++ 3 files changed, 145 insertions(+) create mode 100644 tests/integration/test_distributed_queries_stress/__init__.py create mode 100644 tests/integration/test_distributed_queries_stress/configs/remote_servers.xml create mode 100644 tests/integration/test_distributed_queries_stress/test.py diff --git a/tests/integration/test_distributed_queries_stress/__init__.py b/tests/integration/test_distributed_queries_stress/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml b/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml new file mode 100644 index 00000000000..7d00cebccfc --- /dev/null +++ b/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml @@ -0,0 +1,42 @@ + + 1000 + + + + + true + + node1_r1 + 9000 + + + node1_r2 + 9000 + + + + + + true + + node1_r1 + 9000 + + + node1_r2 + 9000 + + + + + node2_r1 + 9000 + + + node2_r2 + 9000 + + + + + diff --git a/tests/integration/test_distributed_queries_stress/test.py b/tests/integration/test_distributed_queries_stress/test.py new file mode 100644 index 00000000000..dcc4943f7e6 --- /dev/null +++ b/tests/integration/test_distributed_queries_stress/test.py @@ -0,0 +1,103 @@ +# pylint: disable=redefined-outer-name +# pylint: disable=unused-argument +# pylint: disable=line-too-long + +import shlex +import itertools +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1_r1 = cluster.add_instance('node1_r1', main_configs=['configs/remote_servers.xml']) +node2_r1 = cluster.add_instance('node2_r1', main_configs=['configs/remote_servers.xml']) +node1_r2 = cluster.add_instance('node1_r2', main_configs=['configs/remote_servers.xml']) +node2_r2 = cluster.add_instance('node2_r2', main_configs=['configs/remote_servers.xml']) + +def run_benchmark(payload, settings): + node1_r1.exec_in_container([ + 'bash', '-c', 'echo {} | '.format(shlex.quote(payload.strip())) + ' '.join([ + 'clickhouse', 'benchmark', + '--concurrency=100', + '--cumulative', + '--delay=0', + # NOTE: with current matrix even 3 seconds it huge... + '--timelimit=3', + # tune some basic timeouts + '--hedged_connection_timeout_ms=200', + '--connect_timeout_with_failover_ms=200', + '--connections_with_failover_max_tries=5', + *settings, + ]) + ]) + +@pytest.fixture(scope='module') +def started_cluster(): + try: + cluster.start() + + for _, instance in cluster.instances.items(): + instance.query(""" + create table if not exists data ( + key Int, + /* just to increase block size */ + v1 UInt64, + v2 UInt64, + v3 UInt64, + v4 UInt64, + v5 UInt64, + v6 UInt64, + v7 UInt64, + v8 UInt64, + v9 UInt64, + v10 UInt64, + v11 UInt64, + v12 UInt64 + ) Engine=MergeTree() order by key partition by key%5; + insert into data (key) select * from numbers(10); + + create table if not exists dist_one as data engine=Distributed(one_shard, currentDatabase(), data, key); + create table if not exists dist_one_over_dist as data engine=Distributed(one_shard, currentDatabase(), dist_one, yandexConsistentHash(key, 2)); + + create table if not exists dist_two as data engine=Distributed(two_shards, currentDatabase(), data, key); + create table if not exists dist_two_over_dist as data engine=Distributed(two_shards, currentDatabase(), dist_two, yandexConsistentHash(key, 2)); + """) + yield cluster + finally: + cluster.shutdown() + +# since it includes started_cluster fixture at first start +@pytest.mark.timeout(60) +@pytest.mark.parametrize('table,settings', itertools.product( + [ # tables + 'dist_one', + 'dist_one_over_dist', + 'dist_two', + 'dist_two_over_dist', + ], + [ # settings + *list(itertools.combinations([ + '', # defaults + '--prefer_localhost_replica=0', + '--async_socket_for_remote=0', + '--use_hedged_requests=0', + '--optimize_skip_unused_shards=1', + '--distributed_group_by_no_merge=2', + '--optimize_distributed_group_by_sharding_key=1', + + # TODO: enlarge test matrix (but first those values to accept ms): + # + # - sleep_in_send_tables_status + # - sleep_in_send_data + ], 2)) + # TODO: more combinations that just 2 + ], +)) +def test_stress_distributed(table, settings, started_cluster): + payload = f''' + select * from {table} where key = 0; + select * from {table} where key = 1; + select * from {table} where key = 2; + select * from {table} where key = 3; + select * from {table}; + ''' + run_benchmark(payload, settings) From db505553a496b7d51ce73a8b078e9062866d7f3a Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 10:09:51 +0300 Subject: [PATCH 069/266] Reads file as String - description of the new function. --- docs/en/sql-reference/functions/files.md | 35 ++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 docs/en/sql-reference/functions/files.md diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md new file mode 100644 index 00000000000..3fb89db929d --- /dev/null +++ b/docs/en/sql-reference/functions/files.md @@ -0,0 +1,35 @@ +--- +toc_priority: 68 +toc_title: Files +--- + +# Functions for Working with Files {#functions-for-working-with-files} + +## file() {#file} + +Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. + +**Syntax** + +``` sql +file(filename) +``` + +**Arguments** + +- `filename` — The name of the file to read. The file must be located in the user's directory specified in `user_files_path` settings. + +**Example** + +Inserting data from files a.txt and b.txt in the table as separate rows. + +Query: + +``` sql +INSERT INTO table SELECT file('a.txt'), file('b.txt') +``` + +**See Also** + +- [user_files_path][../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path](#) + From 9f6ad165c84270166fa43b15ef0a19be4c040e9d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 11:22:43 +0300 Subject: [PATCH 070/266] Updated description --- docs/en/sql-reference/functions/files.md | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 3fb89db929d..a0da5a2c82d 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -5,23 +5,23 @@ toc_title: Files # Functions for Working with Files {#functions-for-working-with-files} -## file() {#file} +## file(path) {#file} -Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. +Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. **Syntax** ``` sql -file(filename) +file(path) ``` **Arguments** -- `filename` — The name of the file to read. The file must be located in the user's directory specified in `user_files_path` settings. +- `path` — The relative path to the file from `user_files_path`. **Example** -Inserting data from files a.txt and b.txt in the table as separate rows. +Inserting data from files a.txt and b.txt into a table as different rows. Query: @@ -31,5 +31,6 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt') **See Also** -- [user_files_path][../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path](#) +- [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) +- [file](../table-functions/file/) From 25052fc3f2a648efd268283c419ef218b16cdac8 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 11:47:41 +0300 Subject: [PATCH 071/266] Reads file as a String --- docs/en/sql-reference/functions/files.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index a0da5a2c82d..55ecf7d3aba 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -1,5 +1,5 @@ --- -toc_priority: 68 +toc_priority: 43 toc_title: Files --- @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from `user_files_path`. +- `path` — The relative path to the file from `user_files_path`. Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** From e4bf63f4704e347bf48f0df859e7035921784407 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 12:25:33 +0300 Subject: [PATCH 072/266] Link for see also --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 55ecf7d3aba..47b07d9ac9d 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -32,5 +32,5 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt') **See Also** - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) -- [file](../table-functions/file/) +- [file](../table-functions/file.md) From 9f2f0d1095b16f78b40a67054a94db25d62bbfe3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Mar 2021 19:31:00 +0300 Subject: [PATCH 073/266] Refactored hierarchy dictionaries interface --- src/Databases/DatabaseAtomic.cpp | 2 +- src/Databases/DatabaseWithDictionaries.cpp | 2 +- src/Dictionaries/CacheDictionary.cpp | 196 +--- src/Dictionaries/CacheDictionary.h | 28 +- .../ComplexKeyHashedDictionary.cpp | 594 ----------- src/Dictionaries/ComplexKeyHashedDictionary.h | 185 ---- .../DictionaryBlockInputStream.cpp | 200 ++++ src/Dictionaries/DictionaryBlockInputStream.h | 226 +---- src/Dictionaries/DictionaryHelpers.h | 3 +- src/Dictionaries/DictionaryStructure.cpp | 15 +- src/Dictionaries/DictionaryStructure.h | 2 + src/Dictionaries/DirectDictionary.cpp | 202 +--- src/Dictionaries/DirectDictionary.h | 52 +- src/Dictionaries/FlatDictionary.cpp | 201 ++-- src/Dictionaries/FlatDictionary.h | 34 +- src/Dictionaries/HashedDictionary.cpp | 929 +++++++++--------- src/Dictionaries/HashedDictionary.h | 210 ++-- .../HierarchyDictionariesUtils.cpp | 150 +++ src/Dictionaries/HierarchyDictionariesUtils.h | 197 ++++ src/Dictionaries/IDictionary.h | 98 +- src/Dictionaries/IPAddressDictionary.cpp | 9 +- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/PolygonDictionary.cpp | 3 +- src/Dictionaries/PolygonDictionary.h | 2 +- .../RangeDictionaryBlockInputStream.h | 6 +- src/Dictionaries/RangeHashedDictionary.cpp | 22 +- src/Dictionaries/RangeHashedDictionary.h | 12 +- src/Dictionaries/registerDictionaries.cpp | 1 - src/Dictionaries/ya.make | 3 +- src/Functions/FunctionsExternalDictionaries.h | 222 +---- .../ExternalDictionariesLoader.cpp | 4 +- src/Interpreters/ExternalDictionariesLoader.h | 2 +- .../System/StorageSystemDictionaries.cpp | 2 +- ...765_hashed_dictionary_simple_key.reference | 132 +++ .../01765_hashed_dictionary_simple_key.sql | 207 ++++ ...66_hashed_dictionary_complex_key.reference | 56 ++ .../01766_hashed_dictionary_complex_key.sql | 98 ++ 37 files changed, 1898 insertions(+), 2411 deletions(-) delete mode 100644 src/Dictionaries/ComplexKeyHashedDictionary.cpp delete mode 100644 src/Dictionaries/ComplexKeyHashedDictionary.h create mode 100644 src/Dictionaries/DictionaryBlockInputStream.cpp create mode 100644 src/Dictionaries/HierarchyDictionariesUtils.cpp create mode 100644 src/Dictionaries/HierarchyDictionariesUtils.h create mode 100644 tests/queries/0_stateless/01765_hashed_dictionary_simple_key.reference create mode 100644 tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql create mode 100644 tests/queries/0_stateless/01766_hashed_dictionary_complex_key.reference create mode 100644 tests/queries/0_stateless/01766_hashed_dictionary_complex_key.sql diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index e0078da57b7..b4222a7e349 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -567,7 +567,7 @@ void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name auto result = external_loader.getLoadResult(toString(old_name.uuid)); if (!result.object) return; - const auto & dict = dynamic_cast(*result.object); + const auto & dict = dynamic_cast(*result.object); dict.updateDictionaryName(new_name); } void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid) diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index d92f0f1897e..55b04f27c58 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -49,7 +49,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, /// Attach the dictionary as table too. try { - /// TODO Make StorageDictionary an owner of IDictionaryBase objects. + /// TODO Make StorageDictionary an owner of IDictionary objects. /// All DDL operations with dictionaries will work with StorageDictionary table, /// and StorageDictionary will be responsible for loading of DDL dictionaries. /// ExternalLoaderDatabaseConfigRepository and other hacks related to ExternalLoader diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index eedf4dd3d87..2c9d6ca764d 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -13,7 +13,9 @@ #include #include #include + #include +#include namespace ProfileEvents { @@ -39,7 +41,6 @@ namespace DB namespace ErrorCodes { extern const int CACHE_DICTIONARY_UPDATE_FAIL; - extern const int TYPE_MISMATCH; extern const int UNSUPPORTED_METHOD; } @@ -70,8 +71,6 @@ CacheDictionary::CacheDictionary( { if (!source_ptr->supportsSelectiveLoad()) throw Exception{full_name + ": source cannot be used with CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD}; - - setupHierarchicalAttribute(); } template @@ -120,164 +119,6 @@ const IDictionarySource * CacheDictionary::getSource() cons return source_ptr.get(); } -template -void CacheDictionary::toParent(const PaddedPODArray & ids [[maybe_unused]], PaddedPODArray & out [[maybe_unused]]) const -{ - if constexpr (dictionary_key_type == DictionaryKeyType::simple) - { - /// Run update on requested keys before fetch from storage - const auto & attribute_name = hierarchical_attribute->name; - - auto result_type = std::make_shared(); - auto input_column = result_type->createColumn(); - auto & input_column_typed = assert_cast &>(*input_column); - auto & data = input_column_typed.getData(); - data.insert(ids.begin(), ids.end()); - - auto column = getColumn({attribute_name}, result_type, {std::move(input_column)}, {result_type}, {nullptr}); - const auto & result_column_typed = assert_cast &>(*column); - const auto & result_data = result_column_typed.getData(); - - out.assign(result_data); - } - else - throw Exception("Hierarchy is not supported for complex key CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD); -} - - -/// Allow to use single value in same way as array. -static inline UInt64 getAt(const PaddedPODArray & arr, const size_t idx) -{ - return arr[idx]; -} -static inline UInt64 getAt(const UInt64 & value, const size_t) -{ - return value; -} - -template -template -void CacheDictionary::isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const -{ - /// Transform all children to parents until ancestor id or null_value will be reached. - - size_t out_size = out.size(); - memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" - - const auto null_value = hierarchical_attribute->null_value.get(); - - PaddedPODArray children(out_size, 0); - PaddedPODArray parents(child_ids.begin(), child_ids.end()); - - for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) - { - size_t out_idx = 0; - size_t parents_idx = 0; - size_t new_children_idx = 0; - - while (out_idx < out_size) - { - /// Already calculated - if (out[out_idx] != 0xFF) - { - ++out_idx; - continue; - } - - /// No parent - if (parents[parents_idx] == null_value) - { - out[out_idx] = 0; - } - /// Found ancestor - else if (parents[parents_idx] == getAt(ancestor_ids, parents_idx)) - { - out[out_idx] = 1; - } - /// Loop detected - else if (children[new_children_idx] == parents[parents_idx]) - { - out[out_idx] = 1; - } - /// Found intermediate parent, add this value to search at next loop iteration - else - { - children[new_children_idx] = parents[parents_idx]; - ++new_children_idx; - } - - ++out_idx; - ++parents_idx; - } - - if (new_children_idx == 0) - break; - - /// Transform all children to its parents. - children.resize(new_children_idx); - parents.resize(new_children_idx); - - toParent(children, parents); - } -} - -template -void CacheDictionary::isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_ids, out); -} - -template -void CacheDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const UInt64 ancestor_id, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_id, out); -} - -template -void CacheDictionary::isInConstantVector(const UInt64 child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - /// Special case with single child value. - - const auto null_value = hierarchical_attribute->null_value.get(); - - PaddedPODArray child(1, child_id); - PaddedPODArray parent(1); - std::vector ancestors(1, child_id); - - /// Iteratively find all ancestors for child. - for (size_t i = 0; i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) - { - toParent(child, parent); - - if (parent[0] == null_value) - break; - - child[0] = parent[0]; - ancestors.push_back(parent[0]); - } - - /// Assuming short hierarchy, so linear search is Ok. - for (size_t i = 0, out_size = out.size(); i < out_size; ++i) - out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); -} - -template -void CacheDictionary::setupHierarchicalAttribute() -{ - /// TODO: Move this to DictionaryStructure - for (const auto & attribute : dict_struct.attributes) - { - if (attribute.hierarchical) - { - hierarchical_attribute = &attribute; - - if (attribute.underlying_type != AttributeUnderlyingType::utUInt64) - throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; - } - } -} - template ColumnPtr CacheDictionary::getColumn( const std::string & attribute_name, @@ -526,6 +367,32 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k return result; } +template +ColumnPtr CacheDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const +{ + if (dictionary_key_type == DictionaryKeyType::simple) + { + auto result = getHierarchyDefaultImplementation(this, key_column, key_type); + query_count.fetch_add(key_column->size(), std::memory_order_relaxed); + return result; + } + else + return nullptr; +} + +template +ColumnUInt8::Ptr CacheDictionary::isInHierarchy(ColumnPtr key_column, ColumnPtr in_key_column, const DataTypePtr & key_type) const +{ + if (dictionary_key_type == DictionaryKeyType::simple) + { + auto result = isInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); + query_count.fetch_add(key_column->size(), std::memory_order_relaxed); + return result; + } + else + return nullptr; +} + template MutableColumns CacheDictionary::aggregateColumnsInOrderOfKeys( const PaddedPODArray & keys, @@ -618,19 +485,18 @@ MutableColumns CacheDictionary::aggregateColumns( template BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const { - using BlockInputStreamType = DictionaryBlockInputStream; - std::shared_ptr stream; + std::shared_ptr stream; { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; if constexpr (dictionary_key_type == DictionaryKeyType::simple) - stream = std::make_shared(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names); + stream = std::make_shared(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names); else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - stream = std::make_shared(shared_from_this(), max_block_size, keys, column_names); + stream = std::make_shared(shared_from_this(), max_block_size, keys, column_names); } } diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 1192db73737..35ea17abf27 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -130,23 +130,14 @@ public: std::exception_ptr getLastException() const override; - bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && hierarchical_attribute; } + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } - void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; - void isInVectorVector( - const PaddedPODArray & child_ids, - const PaddedPODArray & ancestor_ids, - PaddedPODArray & out) const override; - - void isInVectorConstant( - const PaddedPODArray & child_ids, - const UInt64 ancestor_id, PaddedPODArray & out) const override; - - void isInConstantVector( - const UInt64 child_id, - const PaddedPODArray & ancestor_ids, - PaddedPODArray & out) const override; + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; private: using FetchResult = std::conditional_t; @@ -171,8 +162,6 @@ private: const MutableColumns & fetched_columns_during_update, const HashMap & found_keys_to_fetched_columns_during_update_index); - void setupHierarchicalAttribute(); - void update(CacheDictionaryUpdateUnitPtr update_unit_ptr); /// Update dictionary source pointer if required and return it. Thread safe. @@ -193,9 +182,6 @@ private: return source_ptr; } - template - void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - const DictionaryStructure dict_struct; /// Dictionary source should be used with mutex @@ -218,8 +204,6 @@ private: /// readers. Surprisingly this lock is also used for last_exception pointer. mutable std::shared_mutex rw_lock; - const DictionaryAttribute * hierarchical_attribute = nullptr; - mutable std::exception_ptr last_exception; mutable std::atomic error_count {0}; mutable std::atomic backoff_end_time{std::chrono::system_clock::time_point{}}; diff --git a/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/src/Dictionaries/ComplexKeyHashedDictionary.cpp deleted file mode 100644 index 4086082e66d..00000000000 --- a/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ /dev/null @@ -1,594 +0,0 @@ -#include "ComplexKeyHashedDictionary.h" -#include -#include -#include -#include -#include -#include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" - -namespace DB -{ -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; - extern const int BAD_ARGUMENTS; - extern const int DICTIONARY_IS_EMPTY; -} - -ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( - const StorageID & dict_id_, - const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_, - bool require_nonempty_, - BlockPtr saved_block_) - : IDictionaryBase(dict_id_) - , dict_struct(dict_struct_) - , source_ptr{std::move(source_ptr_)} - , dict_lifetime(dict_lifetime_) - , require_nonempty(require_nonempty_) - , saved_block{std::move(saved_block_)} -{ - createAttributes(); - loadData(); - calculateBytesAllocated(); -} - -ColumnPtr ComplexKeyHashedDictionary::getColumn( - const std::string & attribute_name, - const DataTypePtr & result_type, - const Columns & key_columns, - const DataTypes & key_types, - const ColumnPtr & default_values_column) const -{ - dict_struct.validateKeyTypes(key_types); - - ColumnPtr result; - - const auto & attribute = getAttribute(attribute_name); - const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); - - auto keys_size = key_columns.front()->size(); - - ColumnUInt8::MutablePtr col_null_map_to; - ColumnUInt8::Container * vec_null_map_to = nullptr; - if (attribute.is_nullable) - { - 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; - - const auto attribute_null_value = std::get(attribute.null_values); - AttributeType null_value = static_cast(attribute_null_value); - DictionaryDefaultValueExtractor default_value_extractor(std::move(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, - key_columns, - [&](const size_t row, const StringRef value, bool is_null) - { - if (attribute.is_nullable) - (*vec_null_map_to)[row] = is_null; - - out->insertData(value.data, value.size); - }, - default_value_extractor); - } - else - { - auto & out = column->getData(); - - getItemsImpl( - attribute, - key_columns, - [&](const size_t row, const auto value, bool is_null) - { - if (attribute.is_nullable) - (*vec_null_map_to)[row] = is_null; - - out[row] = value; - }, - default_value_extractor); - } - - result = std::move(column); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - if (attribute.is_nullable) - { - result = ColumnNullable::create(result, std::move(col_null_map_to)); - } - - return result; -} - -ColumnUInt8::Ptr ComplexKeyHashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const -{ - dict_struct.validateKeyTypes(key_types); - - auto size = key_columns.front()->size(); - auto result = ColumnUInt8::create(size); - auto& out = result->getData(); - - const auto & attribute = attributes.front(); - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - - has(attribute, key_columns, out); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - return result; -} - -void ComplexKeyHashedDictionary::createAttributes() -{ - const auto size = dict_struct.attributes.size(); - attributes.reserve(size); - - for (const auto & attribute : dict_struct.attributes) - { - attribute_index_by_name.emplace(attribute.name, attributes.size()); - attributes.push_back(createAttribute(attribute, attribute.null_value)); - - if (attribute.hierarchical) - throw Exception{full_name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), - ErrorCodes::TYPE_MISMATCH}; - } -} - -void ComplexKeyHashedDictionary::blockToAttributes(const Block & block) -{ - /// created upfront to avoid excess allocations - const auto keys_size = dict_struct.key->size(); - StringRefs keys(keys_size); - - const auto attributes_size = attributes.size(); - const auto rows = block.rows(); - element_count += rows; - - const auto key_column_ptrs = ext::map( - ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); - - const auto attribute_column_ptrs = ext::map(ext::range(0, attributes_size), [&](const size_t attribute_idx) - { - return block.safeGetByPosition(keys_size + attribute_idx).column; - }); - - for (const auto row_idx : ext::range(0, rows)) - { - /// calculate key once per row - const auto key = placeKeysInPool(row_idx, key_column_ptrs, keys, keys_pool); - - auto should_rollback = false; - - for (const auto attribute_idx : ext::range(0, attributes_size)) - { - const auto & attribute_column = *attribute_column_ptrs[attribute_idx]; - auto & attribute = attributes[attribute_idx]; - const auto inserted = setAttributeValue(attribute, key, attribute_column[row_idx]); - if (!inserted) - should_rollback = true; - } - - /// @note on multiple equal keys the mapped value for the first one is stored - if (should_rollback) - keys_pool.rollback(key.size); - } -} - -void ComplexKeyHashedDictionary::updateData() -{ - /// created upfront to avoid excess allocations - const auto keys_size = dict_struct.key->size(); - StringRefs keys(keys_size); - - const auto attributes_size = attributes.size(); - - if (!saved_block || saved_block->rows() == 0) - { - auto stream = source_ptr->loadUpdatedAll(); - stream->readPrefix(); - - while (const auto block = stream->read()) - { - /// We are using this method to keep saved data if input stream consists of multiple blocks - if (!saved_block) - saved_block = std::make_shared(block.cloneEmpty()); - for (const auto attribute_idx : ext::range(0, keys_size + attributes_size)) - { - const IColumn & update_column = *block.getByPosition(attribute_idx).column.get(); - MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable(); - saved_column->insertRangeFrom(update_column, 0, update_column.size()); - } - } - stream->readSuffix(); - } - else - { - auto stream = source_ptr->loadUpdatedAll(); - - stream->readPrefix(); - while (Block block = stream->read()) - { - const auto saved_key_column_ptrs = ext::map( - ext::range(0, keys_size), [&](const size_t key_idx) { return saved_block->safeGetByPosition(key_idx).column; }); - - const auto update_key_column_ptrs = ext::map( - ext::range(0, keys_size), [&](const size_t key_idx) { return block.safeGetByPosition(key_idx).column; }); - - Arena temp_key_pool; - ContainerType> update_key_hash; - - for (size_t i = 0; i < block.rows(); ++i) - { - const auto u_key = placeKeysInPool(i, update_key_column_ptrs, keys, temp_key_pool); - update_key_hash[u_key].push_back(i); - } - - const size_t rows = saved_block->rows(); - IColumn::Filter filter(rows); - - for (size_t i = 0; i < saved_block->rows(); ++i) - { - const auto s_key = placeKeysInPool(i, saved_key_column_ptrs, keys, temp_key_pool); - auto * it = update_key_hash.find(s_key); - if (it) - filter[i] = 0; - else - filter[i] = 1; - } - - auto block_columns = block.mutateColumns(); - for (const auto attribute_idx : ext::range(0, keys_size + attributes_size)) - { - auto & column = saved_block->safeGetByPosition(attribute_idx).column; - const auto & filtered_column = column->filter(filter, -1); - - block_columns[attribute_idx]->insertRangeFrom(*filtered_column.get(), 0, filtered_column->size()); - } - - saved_block->setColumns(std::move(block_columns)); - } - stream->readSuffix(); - } - - if (saved_block) - blockToAttributes(*saved_block.get()); -} - -void ComplexKeyHashedDictionary::loadData() -{ - if (!source_ptr->hasUpdateField()) - { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); - - while (const auto block = stream->read()) - blockToAttributes(block); - - stream->readSuffix(); - } - else - updateData(); - - if (require_nonempty && 0 == element_count) - throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY}; -} - -template -void ComplexKeyHashedDictionary::addAttributeSize(const Attribute & attribute) -{ - const auto & map_ref = std::get>(attribute.maps); - bytes_allocated += sizeof(ContainerType) + map_ref.getBufferSizeInBytes(); - bucket_count = map_ref.getBufferSizeInCells(); -} - -template <> -void ComplexKeyHashedDictionary::addAttributeSize(const Attribute & attribute) -{ - const auto & map_ref = std::get>(attribute.maps); - bytes_allocated += sizeof(ContainerType) + map_ref.getBufferSizeInBytes(); - bucket_count = map_ref.getBufferSizeInCells(); - bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); -} - -void ComplexKeyHashedDictionary::calculateBytesAllocated() -{ - bytes_allocated += attributes.size() * sizeof(attributes.front()); - - for (const auto & attribute : attributes) - { - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - - addAttributeSize(attribute); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - } - - bytes_allocated += keys_pool.size(); -} - -template -void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) -{ - attribute.null_values = T(null_value.get()); - attribute.maps.emplace>(); -} - -template <> -void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) -{ - attribute.string_arena = std::make_unique(); - const String & string = null_value.get(); - const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - attribute.null_values.emplace(string_in_arena, string.size()); - attribute.maps.emplace>(); -} - -ComplexKeyHashedDictionary::Attribute -ComplexKeyHashedDictionary::createAttribute(const DictionaryAttribute & attribute, const Field & null_value) -{ - auto nullable_set = attribute.is_nullable ? std::make_unique() : nullptr; - Attribute attr{attribute.underlying_type, attribute.is_nullable, std::move(nullable_set), {}, {}, {}}; - - auto type_call = [&](const auto &dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - createAttributeImpl(attr, null_value); - }; - - callOnDictionaryAttributeType(attribute.underlying_type, type_call); - - return attr; -} - - -template -void ComplexKeyHashedDictionary::getItemsImpl( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const -{ - const auto & attr = std::get>(attribute.maps); - - const auto keys_size = key_columns.size(); - StringRefs keys(keys_size); - Arena temporary_keys_pool; - - const auto rows = key_columns.front()->size(); - for (const auto i : ext::range(0, rows)) - { - /// copy key data to arena so it is contiguous and return StringRef to it - const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool); - - const auto it = attr.find(key); - - if (it) - { - set_value(i, static_cast(it->getMapped()), false); - } - else - { - if (attribute.is_nullable && attribute.nullable_set->find(key) != nullptr) - set_value(i, default_value_extractor[i], true); - else - set_value(i, default_value_extractor[i], false); - } - - /// free memory allocated for the key - temporary_keys_pool.rollback(key.size); - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} - - -template -bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value) -{ - auto & map = std::get>(attribute.maps); - const auto pair = map.insert({key, value}); - return pair.second; -} - -template <> -bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const String value) -{ - const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size()); - return setAttributeValueImpl(attribute, key, StringRef{string_in_arena, value.size()}); -} - -bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const StringRef key, const Field & value) -{ - bool result = false; - - auto type_call = [&](const auto &dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - - if (attribute.is_nullable) - { - if (value.isNull()) - { - attribute.nullable_set->insert(key); - result = true; - return; - } - else - { - attribute.nullable_set->erase(key); - } - } - - result = setAttributeValueImpl(attribute, key, value.get()); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - return result; -} - -const ComplexKeyHashedDictionary::Attribute & ComplexKeyHashedDictionary::getAttribute(const std::string & attribute_name) const -{ - const auto it = attribute_index_by_name.find(attribute_name); - if (it == std::end(attribute_index_by_name)) - throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - - return attributes[it->second]; -} - -StringRef ComplexKeyHashedDictionary::placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool) -{ - const auto keys_size = key_columns.size(); - size_t sum_keys_size{}; - - const char * block_start = nullptr; - for (size_t j = 0; j < keys_size; ++j) - { - keys[j] = key_columns[j]->serializeValueIntoArena(row, pool, block_start); - sum_keys_size += keys[j].size; - } - - const auto * key_start = block_start; - for (size_t j = 0; j < keys_size; ++j) - { - keys[j].data = key_start; - key_start += keys[j].size; - } - - return {block_start, sum_keys_size}; -} - -template -void ComplexKeyHashedDictionary::has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const -{ - const auto & attr = std::get>(attribute.maps); - const auto keys_size = key_columns.size(); - StringRefs keys(keys_size); - Arena temporary_keys_pool; - const auto rows = key_columns.front()->size(); - - for (const auto i : ext::range(0, rows)) - { - /// copy key data to arena so it is contiguous and return StringRef to it - const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool); - - const auto it = attr.find(key); - out[i] = static_cast(it); - - if (attribute.is_nullable && !out[i]) - out[i] = attribute.nullable_set->find(key) != nullptr; - - /// free memory allocated for the key - temporary_keys_pool.rollback(key.size); - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} - -std::vector ComplexKeyHashedDictionary::getKeys() const -{ - const Attribute & attribute = attributes.front(); - - std::vector result; - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - - if constexpr (std::is_same_v) - { - result = getKeys(attribute); - } - else - { - result = getKeys(attribute); - } - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - return result; -} - -template -std::vector ComplexKeyHashedDictionary::getKeys(const Attribute & attribute) const -{ - const ContainerType & attr = std::get>(attribute.maps); - std::vector keys; - keys.reserve(attr.size()); - for (const auto & key : attr) - keys.push_back(key.getKey()); - - if (attribute.is_nullable) - { - for (const auto & key: *attribute.nullable_set) - keys.push_back(key.getKey()); - } - - return keys; -} - -BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const -{ - using BlockInputStreamType = DictionaryBlockInputStream; - auto vector_keys = getKeys(); - - PaddedPODArray keys; - keys.reserve(vector_keys.size()); - keys.assign(vector_keys.begin(), vector_keys.end()); - - return std::make_shared(shared_from_this(), max_block_size, keys, column_names); -} - -void registerDictionaryComplexKeyHashed(DictionaryFactory & factory) -{ - auto create_layout = [=](const std::string &, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr) -> DictionaryPtr - { - if (!dict_struct.key) - throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; - - const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); - const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; - const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); - }; - factory.registerLayout("complex_key_hashed", create_layout, true); -} - -} diff --git a/src/Dictionaries/ComplexKeyHashedDictionary.h b/src/Dictionaries/ComplexKeyHashedDictionary.h deleted file mode 100644 index 091974bbf43..00000000000 --- a/src/Dictionaries/ComplexKeyHashedDictionary.h +++ /dev/null @@ -1,185 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "IDictionary.h" -#include "IDictionarySource.h" -#include "DictionaryStructure.h" -#include "DictionaryHelpers.h" - -namespace DB -{ - -class ComplexKeyHashedDictionary final : public IDictionaryBase -{ -public: - ComplexKeyHashedDictionary( - const StorageID & dict_id_, - const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - const DictionaryLifetime dict_lifetime_, - bool require_nonempty_, - BlockPtr saved_block_ = nullptr); - - std::string getKeyDescription() const { return key_description; } - - std::string getTypeName() const override { return "ComplexKeyHashed"; } - - size_t getBytesAllocated() const override { return bytes_allocated; } - - size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } - - double getHitRate() const override { return 1.0; } - - size_t getElementCount() const override { return element_count; } - - double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - - std::shared_ptr clone() const override - { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); - } - - const IDictionarySource * getSource() const override { return source_ptr.get(); } - - const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } - - const DictionaryStructure & getStructure() const override { return dict_struct; } - - bool isInjective(const std::string & attribute_name) const override - { - return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; - } - - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; } - - ColumnPtr getColumn( - const std::string& attribute_name, - const DataTypePtr & result_type, - const Columns & key_columns, - const DataTypes & key_types, - const ColumnPtr & default_values_column) const override; - - ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; - -private: - template - using ContainerType = HashMapWithSavedHash; - - using NullableSet = HashSetWithSavedHash; - - struct Attribute final - { - AttributeUnderlyingType type; - bool is_nullable; - std::unique_ptr nullable_set; - - std::variant< - UInt8, - UInt16, - UInt32, - UInt64, - UInt128, - Int8, - Int16, - Int32, - Int64, - Decimal32, - Decimal64, - Decimal128, - Float32, - Float64, - StringRef> - null_values; - std::variant< - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType, - ContainerType> - maps; - std::unique_ptr string_arena; - }; - - void createAttributes(); - - void blockToAttributes(const Block & block); - - void updateData(); - - void loadData(); - - template - void addAttributeSize(const Attribute & attribute); - - void calculateBytesAllocated(); - - template - static void createAttributeImpl(Attribute & attribute, const Field & null_value); - - static Attribute createAttribute(const DictionaryAttribute & attribute, const Field & null_value); - - template - void getItemsImpl( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const; - - template - static bool setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value); - - static bool setAttributeValue(Attribute & attribute, const StringRef key, const Field & value); - - const Attribute & getAttribute(const std::string & attribute_name) const; - - static StringRef placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool); - - template - void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const; - - std::vector getKeys() const; - - template - std::vector getKeys(const Attribute & attribute) const; - - const DictionaryStructure dict_struct; - const DictionarySourcePtr source_ptr; - const DictionaryLifetime dict_lifetime; - const bool require_nonempty; - const std::string key_description{dict_struct.getKeyDescription()}; - - std::map attribute_index_by_name; - std::vector attributes; - Arena keys_pool; - - size_t bytes_allocated = 0; - size_t element_count = 0; - size_t bucket_count = 0; - mutable std::atomic query_count{0}; - - BlockPtr saved_block; -}; - -} diff --git a/src/Dictionaries/DictionaryBlockInputStream.cpp b/src/Dictionaries/DictionaryBlockInputStream.cpp new file mode 100644 index 00000000000..433ff211831 --- /dev/null +++ b/src/Dictionaries/DictionaryBlockInputStream.cpp @@ -0,0 +1,200 @@ +#include "DictionaryBlockInputStream.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +DictionaryBlockInputStream::DictionaryBlockInputStream( + std::shared_ptr dictionary_, UInt64 max_block_size_, PaddedPODArray && ids_, const Names & column_names_) + : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) + , dictionary(dictionary_) + , column_names(column_names_) + , ids(std::move(ids_)) + , key_type(DictionaryInputStreamKeyType::Id) +{ +} + +DictionaryBlockInputStream::DictionaryBlockInputStream( + std::shared_ptr dictionary_, + UInt64 max_block_size_, + const PaddedPODArray & keys, + const Names & column_names_) + : DictionaryBlockInputStreamBase(keys.size(), max_block_size_) + , dictionary(dictionary_) + , column_names(column_names_) + , key_type(DictionaryInputStreamKeyType::ComplexKey) +{ + const DictionaryStructure & dictionary_structure = dictionary->getStructure(); + fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns); +} + +DictionaryBlockInputStream::DictionaryBlockInputStream( + std::shared_ptr dictionary_, + UInt64 max_block_size_, + const Columns & data_columns_, + const Names & column_names_, + GetColumnsFunction && get_key_columns_function_, + GetColumnsFunction && get_view_columns_function_) + : DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_) + , dictionary(dictionary_) + , column_names(column_names_) + , data_columns(data_columns_) + , get_key_columns_function(std::move(get_key_columns_function_)) + , get_view_columns_function(std::move(get_view_columns_function_)) + , key_type(DictionaryInputStreamKeyType::Callback) +{ +} + +Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const +{ + /// TODO: Rewrite + switch (key_type) + { + case DictionaryInputStreamKeyType::ComplexKey: + { + Columns columns; + ColumnsWithTypeAndName view_columns; + columns.reserve(key_columns.size()); + for (const auto & key_column : key_columns) + { + ColumnPtr column = key_column.column->cut(start, length); + columns.emplace_back(column); + view_columns.emplace_back(column, key_column.type, key_column.name); + } + return fillBlock({}, columns, {}, std::move(view_columns)); + } + + case DictionaryInputStreamKeyType::Id: + { + PaddedPODArray ids_to_fill(ids.begin() + start, ids.begin() + start + length); + return fillBlock(ids_to_fill, {}, {}, {}); + } + + case DictionaryInputStreamKeyType::Callback: + { + Columns columns; + columns.reserve(data_columns.size()); + for (const auto & data_column : data_columns) + columns.push_back(data_column->cut(start, length)); + const DictionaryStructure & dictionaty_structure = dictionary->getStructure(); + const auto & attributes = *dictionaty_structure.key; + ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes); + ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes); + DataTypes types; + columns.clear(); + for (const auto & key_column : keys_with_type_and_name) + { + columns.push_back(key_column.column); + types.push_back(key_column.type); + } + return fillBlock({}, columns, types, std::move(view_with_type_and_name)); + } + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected DictionaryInputStreamKeyType."); +} + +Block DictionaryBlockInputStream::fillBlock( + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & types, + ColumnsWithTypeAndName && view) const +{ + std::unordered_set names(column_names.begin(), column_names.end()); + + DataTypes data_types = types; + ColumnsWithTypeAndName block_columns; + + data_types.reserve(keys.size()); + const DictionaryStructure & dictionary_structure = dictionary->getStructure(); + if (data_types.empty() && dictionary_structure.key) + for (const auto & key : *dictionary_structure.key) + data_types.push_back(key.type); + + for (const auto & column : view) + if (names.find(column.name) != names.end()) + block_columns.push_back(column); + + const DictionaryStructure & structure = dictionary->getStructure(); + ColumnPtr ids_column = getColumnFromIds(ids_to_fill); + + if (structure.id && names.find(structure.id->name) != names.end()) + { + block_columns.emplace_back(ids_column, std::make_shared(), structure.id->name); + } + + auto dictionary_key_type = dictionary->getKeyType(); + + for (const auto idx : ext::range(0, structure.attributes.size())) + { + const DictionaryAttribute & attribute = structure.attributes[idx]; + if (names.find(attribute.name) != names.end()) + { + ColumnPtr column; + + if (dictionary_key_type == DictionaryKeyType::simple) + { + column = dictionary->getColumn( + attribute.name, + attribute.type, + {ids_column}, + {std::make_shared()}, + nullptr /* default_values_column */); + } + else + { + column = dictionary->getColumn( + attribute.name, + attribute.type, + keys, + data_types, + nullptr /* default_values_column*/); + } + + block_columns.emplace_back(column, attribute.type, attribute.name); + } + } + + return Block(block_columns); +} + +ColumnPtr DictionaryBlockInputStream::getColumnFromIds(const PaddedPODArray & ids_to_fill) +{ + auto column_vector = ColumnVector::create(); + column_vector->getData().assign(ids_to_fill); + return column_vector; +} + +void DictionaryBlockInputStream::fillKeyColumns( + const PaddedPODArray & keys, + size_t start, + size_t size, + const DictionaryStructure & dictionary_structure, + ColumnsWithTypeAndName & result) +{ + MutableColumns columns; + columns.reserve(dictionary_structure.key->size()); + + for (const DictionaryAttribute & attribute : *dictionary_structure.key) + columns.emplace_back(attribute.type->createColumn()); + + for (auto idx : ext::range(start, size)) + { + const auto & key = keys[idx]; + const auto *ptr = key.data; + for (auto & column : columns) + ptr = column->deserializeAndInsertFromArena(ptr); + } + + for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i) + { + const auto & dictionary_attribute = (*dictionary_structure.key)[i]; + result.emplace_back(ColumnWithTypeAndName{std::move(columns[i]), dictionary_attribute.type, dictionary_attribute.name}); + } +} + +} diff --git a/src/Dictionaries/DictionaryBlockInputStream.h b/src/Dictionaries/DictionaryBlockInputStream.h index 71615efa7f8..5197df411fa 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.h +++ b/src/Dictionaries/DictionaryBlockInputStream.h @@ -16,27 +16,22 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} /// TODO: Remove this class /* BlockInputStream implementation for external dictionaries * read() returns blocks consisting of the in-memory contents of the dictionaries */ -template class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase { public: DictionaryBlockInputStream( - std::shared_ptr dictionary, + std::shared_ptr dictionary, UInt64 max_block_size, - PaddedPODArray && ids, + PaddedPODArray && ids, const Names & column_names); DictionaryBlockInputStream( - std::shared_ptr dictionary, + std::shared_ptr dictionary, UInt64 max_block_size, const PaddedPODArray & keys, const Names & column_names); @@ -48,7 +43,7 @@ public: // and get_view_columns_function to get key representation. // Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string DictionaryBlockInputStream( - std::shared_ptr dictionary, + std::shared_ptr dictionary, UInt64 max_block_size, const Columns & data_columns, const Names & column_names, @@ -61,21 +56,24 @@ protected: Block getBlock(size_t start, size_t length) const override; private: - Block - fillBlock(const PaddedPODArray & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const; + Block fillBlock( + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & types, + ColumnsWithTypeAndName && view) const; - ColumnPtr getColumnFromIds(const PaddedPODArray & ids_to_fill) const; + static ColumnPtr getColumnFromIds(const PaddedPODArray & ids_to_fill); - void fillKeyColumns( + static void fillKeyColumns( const PaddedPODArray & keys, size_t start, size_t size, const DictionaryStructure & dictionary_structure, - ColumnsWithTypeAndName & columns) const; + ColumnsWithTypeAndName & result); - std::shared_ptr dictionary; + std::shared_ptr dictionary; Names column_names; - PaddedPODArray ids; + PaddedPODArray ids; ColumnsWithTypeAndName key_columns; Columns data_columns; @@ -92,200 +90,4 @@ private: DictionaryInputStreamKeyType key_type; }; - -template -DictionaryBlockInputStream::DictionaryBlockInputStream( - std::shared_ptr dictionary_, UInt64 max_block_size_, PaddedPODArray && ids_, const Names & column_names_) - : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) - , dictionary(dictionary_) - , column_names(column_names_) - , ids(std::move(ids_)) - , key_type(DictionaryInputStreamKeyType::Id) -{ -} - -template -DictionaryBlockInputStream::DictionaryBlockInputStream( - std::shared_ptr dictionary_, - UInt64 max_block_size_, - const PaddedPODArray & keys, - const Names & column_names_) - : DictionaryBlockInputStreamBase(keys.size(), max_block_size_) - , dictionary(dictionary_) - , column_names(column_names_) - , key_type(DictionaryInputStreamKeyType::ComplexKey) -{ - const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns); -} - -template -DictionaryBlockInputStream::DictionaryBlockInputStream( - std::shared_ptr dictionary_, - UInt64 max_block_size_, - const Columns & data_columns_, - const Names & column_names_, - GetColumnsFunction && get_key_columns_function_, - GetColumnsFunction && get_view_columns_function_) - : DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_) - , dictionary(dictionary_) - , column_names(column_names_) - , data_columns(data_columns_) - , get_key_columns_function(std::move(get_key_columns_function_)) - , get_view_columns_function(std::move(get_view_columns_function_)) - , key_type(DictionaryInputStreamKeyType::Callback) -{ -} - - -template -Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const -{ - /// TODO: Rewrite - switch (key_type) - { - case DictionaryInputStreamKeyType::ComplexKey: - { - Columns columns; - ColumnsWithTypeAndName view_columns; - columns.reserve(key_columns.size()); - for (const auto & key_column : key_columns) - { - ColumnPtr column = key_column.column->cut(start, length); - columns.emplace_back(column); - view_columns.emplace_back(column, key_column.type, key_column.name); - } - return fillBlock({}, columns, {}, std::move(view_columns)); - } - - case DictionaryInputStreamKeyType::Id: - { - PaddedPODArray ids_to_fill(ids.begin() + start, ids.begin() + start + length); - return fillBlock(ids_to_fill, {}, {}, {}); - } - - case DictionaryInputStreamKeyType::Callback: - { - Columns columns; - columns.reserve(data_columns.size()); - for (const auto & data_column : data_columns) - columns.push_back(data_column->cut(start, length)); - const DictionaryStructure & dictionaty_structure = dictionary->getStructure(); - const auto & attributes = *dictionaty_structure.key; - ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes); - ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes); - DataTypes types; - columns.clear(); - for (const auto & key_column : keys_with_type_and_name) - { - columns.push_back(key_column.column); - types.push_back(key_column.type); - } - return fillBlock({}, columns, types, std::move(view_with_type_and_name)); - } - } - - throw Exception("Unexpected DictionaryInputStreamKeyType.", ErrorCodes::LOGICAL_ERROR); -} - -template -Block DictionaryBlockInputStream::fillBlock( - const PaddedPODArray & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const -{ - std::unordered_set names(column_names.begin(), column_names.end()); - - DataTypes data_types = types; - ColumnsWithTypeAndName block_columns; - - data_types.reserve(keys.size()); - const DictionaryStructure & dictionaty_structure = dictionary->getStructure(); - if (data_types.empty() && dictionaty_structure.key) - for (const auto & key : *dictionaty_structure.key) - data_types.push_back(key.type); - - for (const auto & column : view) - if (names.find(column.name) != names.end()) - block_columns.push_back(column); - - const DictionaryStructure & structure = dictionary->getStructure(); - ColumnPtr ids_column = getColumnFromIds(ids_to_fill); - - if (structure.id && names.find(structure.id->name) != names.end()) - { - block_columns.emplace_back(ids_column, std::make_shared(), structure.id->name); - } - - auto dictionary_key_type = dictionary->getKeyType(); - - for (const auto idx : ext::range(0, structure.attributes.size())) - { - const DictionaryAttribute & attribute = structure.attributes[idx]; - if (names.find(attribute.name) != names.end()) - { - ColumnPtr column; - - if (dictionary_key_type == DictionaryKeyType::simple) - { - column = dictionary->getColumn( - attribute.name, - attribute.type, - {ids_column}, - {std::make_shared()}, - nullptr /* default_values_column */); - } - else - { - column = dictionary->getColumn( - attribute.name, - attribute.type, - keys, - data_types, - nullptr /* default_values_column*/); - } - - block_columns.emplace_back(column, attribute.type, attribute.name); - } - } - - return Block(block_columns); -} - -template -ColumnPtr DictionaryBlockInputStream::getColumnFromIds(const PaddedPODArray & ids_to_fill) const -{ - auto column_vector = ColumnVector::create(); - column_vector->getData().reserve(ids_to_fill.size()); - for (UInt64 id : ids_to_fill) - column_vector->insertValue(id); - return column_vector; -} - - -template -void DictionaryBlockInputStream::fillKeyColumns( - const PaddedPODArray & keys, - size_t start, - size_t size, - const DictionaryStructure & dictionary_structure, - ColumnsWithTypeAndName & res) const -{ - MutableColumns columns; - columns.reserve(dictionary_structure.key->size()); - - for (const DictionaryAttribute & attribute : *dictionary_structure.key) - columns.emplace_back(attribute.type->createColumn()); - - for (auto idx : ext::range(start, size)) - { - const auto & key = keys[idx]; - const auto *ptr = key.data; - for (auto & column : columns) - ptr = column->deserializeAndInsertFromArena(ptr); - } - - for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i) - res.emplace_back( - ColumnWithTypeAndName{std::move(columns[i]), (*dictionary_structure.key)[i].type, (*dictionary_structure.key)[i].name}); -} - } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 5fda5f2599e..bb0eba40159 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -370,9 +370,10 @@ private: * If column is constant parameter backup_storage is used to store values. */ +/// TODO: Remove template static const PaddedPODArray & getColumnVectorData( - const IDictionaryBase * dictionary, + const IDictionary * dictionary, const ColumnPtr column, PaddedPODArray & backup_storage) { diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index ea3e3efa03d..aa7423cbe92 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -200,8 +200,21 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration for (size_t i = 0; i < attributes.size(); ++i) { - const auto & attribute_name = attributes[i].name; + const auto & attribute = attributes[i]; + const auto & attribute_name = attribute.name; attribute_name_to_index[attribute_name] = i; + + if (attribute.hierarchical) + { + if (id && attribute.underlying_type != AttributeUnderlyingType::utUInt64) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Hierarchical attribute type for dictionary with simple key must be UInt64. Actual ({})", + toString(attribute.underlying_type)); + else if (key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy"); + + hierarchical_attribute_index = i; + } } if (attributes.empty()) diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 39332f2dff2..419e90ac3db 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -152,6 +152,8 @@ struct DictionaryStructure final std::unordered_map attribute_name_to_index; std::optional range_min; std::optional range_max; + std::optional hierarchical_attribute_index; + bool has_expressions = false; bool access_to_key_from_attributes = false; diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 4cb9e0cd629..5f03dd44ee7 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -1,158 +1,33 @@ #include "DirectDictionary.h" -#include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include + +#include +#include namespace DB { namespace ErrorCodes { - extern const int TYPE_MISMATCH; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; } -namespace -{ - - inline UInt64 getAt(const PaddedPODArray & arr, const size_t idx) - { - return arr[idx]; - } - - inline UInt64 getAt(const UInt64 & value, const size_t) - { - return value; - } - -} - template DirectDictionary::DirectDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - BlockPtr saved_block_) + DictionarySourcePtr source_ptr_) : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} - , saved_block{std::move(saved_block_)} { if (!source_ptr->supportsSelectiveLoad()) throw Exception{full_name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; - - setup(); -} - -template -void DirectDictionary::toParent(const PaddedPODArray & ids [[maybe_unused]], PaddedPODArray & out [[maybe_unused]]) const -{ - if constexpr (dictionary_key_type == DictionaryKeyType::simple) - { - const auto & attribute_name = hierarchical_attribute->name; - - auto result_type = std::make_shared(); - auto input_column = result_type->createColumn(); - auto & input_column_typed = assert_cast &>(*input_column); - auto & data = input_column_typed.getData(); - data.insert(ids.begin(), ids.end()); - - auto column = getColumn({attribute_name}, result_type, {std::move(input_column)}, {result_type}, {nullptr}); - const auto & result_column_typed = assert_cast &>(*column); - const auto & result_data = result_column_typed.getData(); - - out.assign(result_data); - } - else - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Hierarchy is not supported for complex key DirectDictionary"); -} - -template -UInt64 DirectDictionary::getValueOrNullByKey(const Key & to_find) const -{ - std::vector required_key = {to_find}; - - auto stream = source_ptr->loadIds(required_key); - stream->readPrefix(); - - bool is_found = false; - UInt64 result = hierarchical_attribute->null_value.template get(); - - while (const auto block = stream->read()) - { - const IColumn & id_column = *block.safeGetByPosition(0).column; - - for (const size_t attribute_idx : ext::range(0, dict_struct.attributes.size())) - { - if (is_found) - break; - - const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column; - - for (const auto row_idx : ext::range(0, id_column.size())) - { - const auto key = id_column[row_idx].get(); - - if (key == to_find && hierarchical_attribute->name == attribute_name_by_index.at(attribute_idx)) - { - result = attribute_column[row_idx].get(); - is_found = true; - break; - } - } - } - } - - stream->readSuffix(); - - return result; -} - -template -template -void DirectDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const -{ - const auto null_value = hierarchical_attribute->null_value.template get(); - const auto rows = out.size(); - - for (const auto row : ext::range(0, rows)) - { - auto id = getAt(child_ids, row); - const auto ancestor_id = getAt(ancestor_ids, row); - - for (size_t i = 0; id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) - id = getValueOrNullByKey(id); - - out[row] = id != null_value && id == ancestor_id; - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} - -template -void DirectDictionary::isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_ids, out); -} - -template -void DirectDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const UInt64 ancestor_id, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_id, out); -} - -template -void DirectDictionary::isInConstantVector(const UInt64 child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_id, ancestor_ids, out); } template @@ -179,7 +54,7 @@ ColumnPtr DirectDictionary::getColumn( auto fetched_from_storage = attribute.type->createColumn(); size_t fetched_key_index = 0; - size_t requested_attribute_index = attribute_index_by_name.find(attribute_name)->second; + size_t requested_attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; Columns block_key_columns; size_t dictionary_keys_size = dict_struct.getKeysNames().size(); @@ -310,6 +185,37 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & return result; } +template +ColumnPtr DirectDictionary::getHierarchy( + ColumnPtr key_column, + const DataTypePtr & key_type) const +{ + if (dictionary_key_type == DictionaryKeyType::simple) + { + auto result = getHierarchyDefaultImplementation(this, key_column, key_type); + query_count.fetch_add(key_column->size(), std::memory_order_relaxed); + return result; + } + else + return nullptr; +} + +template +ColumnUInt8::Ptr DirectDictionary::isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const +{ + if (dictionary_key_type == DictionaryKeyType::simple) + { + auto result = isInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); + query_count.fetch_add(key_column->size(), std::memory_order_relaxed); + return result; + } + else + return nullptr; +} + template BlockInputStreamPtr DirectDictionary::getSourceBlockInputStream( const Columns & key_columns [[maybe_unused]], @@ -342,32 +248,6 @@ BlockInputStreamPtr DirectDictionary::getSourceBlockInputSt return stream; } -template -void DirectDictionary::setup() -{ - /// TODO: Move this to DictionaryStructure - size_t dictionary_attributes_size = dict_struct.attributes.size(); - for (size_t i = 0; i < dictionary_attributes_size; ++i) - { - const auto & attribute = dict_struct.attributes[i]; - attribute_index_by_name[attribute.name] = i; - attribute_name_by_index[i] = attribute.name; - - if (attribute.hierarchical) - { - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): hierarchical attributes are not supported for complex key direct dictionary", - full_name); - - hierarchical_attribute = &attribute; - - if (attribute.underlying_type != AttributeUnderlyingType::utUInt64) - throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; - } - } -} - template BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const { diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 685fd707ded..6bca6ac6a18 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -18,11 +18,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - template class DirectDictionary final : public IDictionary { @@ -33,8 +28,7 @@ public: DirectDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, - DictionarySourcePtr source_ptr_, - BlockPtr saved_block_ = nullptr); + DictionarySourcePtr source_ptr_); std::string getTypeName() const override { @@ -56,7 +50,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone()); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -67,26 +61,9 @@ public: bool isInjective(const std::string & attribute_name) const override { - auto it = attribute_index_by_name.find(attribute_name); - - if (it == attribute_index_by_name.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "({}): no attribute with name ({}) in dictionary", - full_name, - attribute_name); - - return dict_struct.attributes[it->second].injective; + return dict_struct.getAttribute(attribute_name).injective; } - bool hasHierarchy() const override { return hierarchical_attribute; } - - void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; - - void isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - void isInVectorConstant(const PaddedPODArray & child_ids, const UInt64 ancestor_id, PaddedPODArray & out) const override; - void isInConstantVector(const UInt64 child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - DictionaryKeyType getKeyType() const override { return dictionary_key_type; } ColumnPtr getColumn( @@ -98,30 +75,25 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; + bool hasHierarchy() const override { return dict_struct.hierarchical_attribute_index.has_value(); } + + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; + + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: - void setup(); - BlockInputStreamPtr getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; - UInt64 getValueOrNullByKey(const UInt64 & to_find) const; - - template - void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; - std::unordered_map attribute_index_by_name; - std::unordered_map attribute_name_by_index; - - const DictionaryAttribute * hierarchical_attribute = nullptr; - mutable std::atomic query_count{0}; - - BlockPtr saved_block; }; extern template class DirectDictionary; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index eb63d716913..c67e9686e10 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -1,20 +1,22 @@ #include "FlatDictionary.h" #include +#include + #include #include #include #include #include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int TYPE_MISMATCH; extern const int ARGUMENT_OUT_OF_BOUND; extern const int BAD_ARGUMENTS; extern const int DICTIONARY_IS_EMPTY; @@ -24,7 +26,6 @@ namespace ErrorCodes static const auto initial_array_size = 1024; static const auto max_array_size = 500000; - FlatDictionary::FlatDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, @@ -45,69 +46,6 @@ FlatDictionary::FlatDictionary( calculateBytesAllocated(); } - -void FlatDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const -{ - const auto null_value = std::get(hierarchical_attribute->null_values); - DictionaryDefaultValueExtractor extractor(null_value); - - getItemsImpl( - *hierarchical_attribute, - ids, - [&](const size_t row, const UInt64 value) { out[row] = value; }, - extractor); -} - - -/// Allow to use single value in same way as array. -static inline FlatDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) -{ - return arr[idx]; -} -static inline FlatDictionary::Key getAt(const FlatDictionary::Key & value, const size_t) -{ - return value; -} - -template -void FlatDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const -{ - const auto null_value = std::get(hierarchical_attribute->null_values); - const auto & attr = std::get>(hierarchical_attribute->arrays); - const auto rows = out.size(); - - size_t loaded_size = attr.size(); - for (const auto row : ext::range(0, rows)) - { - auto id = getAt(child_ids, row); - const auto ancestor_id = getAt(ancestor_ids, row); - - for (size_t i = 0; id < loaded_size && id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) - id = attr[id]; - - out[row] = id != null_value && id == ancestor_id; - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} - - -void FlatDictionary::isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_ids, out); -} - -void FlatDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_id, out); -} - -void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_id, ancestor_ids, out); -} - ColumnPtr FlatDictionary::getColumn( const std::string & attribute_name, const DataTypePtr & result_type, @@ -117,14 +55,16 @@ ColumnPtr FlatDictionary::getColumn( { ColumnPtr result; - PaddedPODArray backup_storage; + PaddedPODArray backup_storage; const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage); auto size = ids.size(); - const auto & attribute = getAttribute(attribute_name); const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + const auto & attribute = attributes[attribute_index]; + auto type_call = [&](const auto & dictionary_attribute_type) { using Type = std::decay_t; @@ -183,10 +123,9 @@ ColumnPtr FlatDictionary::getColumn( return result; } - ColumnUInt8::Ptr FlatDictionary::hasKeys(const Columns & key_columns, const DataTypes &) const { - PaddedPODArray backup_storage; + PaddedPODArray backup_storage; const auto& ids = getColumnVectorData(this, key_columns.front(), backup_storage); auto result = ColumnUInt8::create(ext::size(ids)); @@ -205,24 +144,90 @@ ColumnUInt8::Ptr FlatDictionary::hasKeys(const Columns & key_columns, const Data return result; } +ColumnPtr FlatDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &) const +{ + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const UInt64 null_value = std::get(hierarchical_attribute.null_values); + const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); + + auto is_key_valid_func = [&, this](auto & key) + { + return key < loaded_ids.size() && loaded_ids[key]; + }; + + auto get_parent_key_func = [&, this](auto & hierarchy_key) + { + std::optional result; + + if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key]) + return result; + + result = parent_keys[hierarchy_key]; + + return result; + }; + + auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, null_value, is_key_valid_func, get_parent_key_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + + return dictionary_hierarchy_array; +} + +ColumnUInt8::Ptr FlatDictionary::isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr &) const +{ + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + PaddedPODArray keys_in_backup_storage; + const auto & keys_in = getColumnVectorData(this, in_key_column, keys_in_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const UInt64 null_value = std::get(hierarchical_attribute.null_values); + const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); + + auto is_key_valid_func = [&, this](auto & key) + { + return key < loaded_ids.size() && loaded_ids[key]; + }; + + auto get_parent_key_func = [&, this](auto & hierarchy_key) + { + std::optional result; + + if (hierarchy_key >= loaded_ids.size() || !loaded_ids[hierarchy_key]) + return result; + + result = parent_keys[hierarchy_key]; + + return result; + }; + + auto is_in_hierarchy_result = isInKeysHierarchy(keys, keys_in, null_value, is_key_valid_func, get_parent_key_func); + + auto result = ColumnUInt8::create(); + result->getData() = std::move(is_in_hierarchy_result); + + return result; +} + void FlatDictionary::createAttributes() { const auto size = dict_struct.attributes.size(); attributes.reserve(size); for (const auto & attribute : dict_struct.attributes) - { - attribute_index_by_name.emplace(attribute.name, attributes.size()); attributes.push_back(createAttribute(attribute, attribute.null_value)); - - if (attribute.hierarchical) - { - hierarchical_attribute = &attributes.back(); - - if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64) - throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; - } - } } void FlatDictionary::blockToAttributes(const Block & block) @@ -271,7 +276,7 @@ void FlatDictionary::updateData() const auto & saved_id_column = *saved_block->safeGetByPosition(0).column; const auto & update_id_column = *block.safeGetByPosition(0).column; - std::unordered_map> update_ids; + std::unordered_map> update_ids; for (size_t row = 0; row < update_id_column.size(); ++row) { const auto id = update_id_column.get64(row); @@ -280,7 +285,7 @@ void FlatDictionary::updateData() const size_t saved_rows = saved_id_column.size(); IColumn::Filter filter(saved_rows); - std::unordered_map>::iterator it; + std::unordered_map>::iterator it; for (size_t row = 0; row < saved_id_column.size(); ++row) { @@ -385,7 +390,6 @@ void FlatDictionary::createAttributeImpl(Attribute & attribute, const Fi attribute.arrays.emplace>(initial_array_size, StringRef(string_in_arena, string.size())); } - FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value) { auto nullable_set = attribute.is_nullable ? std::make_optional() : std::optional{}; @@ -408,7 +412,7 @@ FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttrib template void FlatDictionary::getItemsImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & ids, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const { @@ -425,7 +429,7 @@ void FlatDictionary::getItemsImpl( } template -void FlatDictionary::resize(Attribute & attribute, const Key id) +void FlatDictionary::resize(Attribute & attribute, const UInt64 id) { if (id >= max_array_size) throw Exception{full_name + ": identifier should be less than " + toString(max_array_size), ErrorCodes::ARGUMENT_OUT_OF_BOUND}; @@ -440,7 +444,7 @@ void FlatDictionary::resize(Attribute & attribute, const Key id) } template -void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T & value) +void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value) { auto & array = std::get>(attribute.arrays); array[id] = value; @@ -448,13 +452,13 @@ void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, } template <> -void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const String & value) +void FlatDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const String & value) { const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size()); setAttributeValueImpl(attribute, id, StringRef{string_in_arena, value.size()}); } -void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, const Field & value) +void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value) { auto type_call = [&](const auto &dictionary_attribute_type) { @@ -484,21 +488,11 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, cons callOnDictionaryAttributeType(attribute.type, type_call); } - -const FlatDictionary::Attribute & FlatDictionary::getAttribute(const std::string & attribute_name) const -{ - const auto it = attribute_index_by_name.find(attribute_name); - if (it == std::end(attribute_index_by_name)) - throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - - return attributes[it->second]; -} - -PaddedPODArray FlatDictionary::getIds() const +PaddedPODArray FlatDictionary::getIds() const { const auto ids_count = ext::size(loaded_ids); - PaddedPODArray ids; + PaddedPODArray ids; ids.reserve(ids_count); for (auto idx : ext::range(0, ids_count)) @@ -509,8 +503,7 @@ PaddedPODArray FlatDictionary::getIds() const BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const { - using BlockInputStreamType = DictionaryBlockInputStream; - return std::make_shared(shared_from_this(), max_block_size, getIds(), column_names); + return std::make_shared(shared_from_this(), max_block_size, getIds(), column_names); } void registerDictionaryFlat(DictionaryFactory & factory) diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index f491eb28641..a47ac8c34d8 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -59,18 +59,9 @@ public: bool isInjective(const std::string & attribute_name) const override { - return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; + return dict_struct.getAttribute(attribute_name).injective; } - bool hasHierarchy() const override { return hierarchical_attribute; } - - void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; - - void isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; - void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; } ColumnPtr getColumn( @@ -82,13 +73,22 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; + bool hasHierarchy() const override { return dict_struct.hierarchical_attribute_index.has_value(); } + + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override; + + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: template using ContainerType = PaddedPODArray; - using NullableSet = HashSet>; + using NullableSet = HashSet>; struct Attribute final { @@ -151,24 +151,24 @@ private: template void getItemsImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & ids, ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; template - void resize(Attribute & attribute, const Key id); + void resize(Attribute & attribute, const UInt64 id); template - void setAttributeValueImpl(Attribute & attribute, const Key id, const T & value); + void setAttributeValueImpl(Attribute & attribute, const UInt64 id, const T & value); - void setAttributeValue(Attribute & attribute, const Key id, const Field & value); + void setAttributeValue(Attribute & attribute, const UInt64 id, const Field & value); const Attribute & getAttribute(const std::string & attribute_name) const; template void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - PaddedPODArray getIds() const; + PaddedPODArray getIds() const; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -177,7 +177,6 @@ private: std::map attribute_index_by_name; std::vector attributes; - const Attribute * hierarchical_attribute = nullptr; std::vector loaded_ids; size_t bytes_allocated = 0; @@ -185,6 +184,7 @@ private: size_t bucket_count = 0; mutable std::atomic query_count{0}; + /// TODO: Remove BlockPtr saved_block; }; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 708be7945f1..b5cb6b43396 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -1,217 +1,169 @@ #include "HashedDictionary.h" + #include -#include "DictionaryBlockInputStream.h" -#include "DictionaryFactory.h" -#include "ClickHouseDictionarySource.h" + #include -#include +#include #include #include -#include +#include -namespace -{ - -/// NOTE: Trailing return type is explicitly specified for SFINAE. - -/// google::sparse_hash_map -template auto first(const T & value) -> decltype(value.first) { return value.first; } // NOLINT -template auto second(const T & value) -> decltype(value.second) { return value.second; } // NOLINT - -/// HashMap -template auto first(const T & value) -> decltype(value.getKey()) { return value.getKey(); } // NOLINT -template auto second(const T & value) -> decltype(value.getMapped()) { return value.getMapped(); } // NOLINT - -} +#include +#include +#include namespace DB { + namespace ErrorCodes { - extern const int TYPE_MISMATCH; extern const int BAD_ARGUMENTS; extern const int DICTIONARY_IS_EMPTY; extern const int UNSUPPORTED_METHOD; } - -HashedDictionary::HashedDictionary( +template +HashedDictionary::HashedDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, - bool sparse_, BlockPtr saved_block_) : IDictionary(dict_id_) , dict_struct(dict_struct_) - , source_ptr{std::move(source_ptr_)} + , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) - , sparse(sparse_) - , saved_block{std::move(saved_block_)} + , saved_block(std::move(saved_block_)) { createAttributes(); loadData(); calculateBytesAllocated(); } - -void HashedDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const -{ - const auto null_value = std::get(hierarchical_attribute->null_values); - DictionaryDefaultValueExtractor extractor(null_value); - - getItemsImpl( - *hierarchical_attribute, - ids, - [&](const size_t row, const UInt64 value) { out[row] = value; }, - extractor); -} - - -/// Allow to use single value in same way as array. -static inline HashedDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) -{ - return arr[idx]; -} -static inline HashedDictionary::Key getAt(const HashedDictionary::Key & value, const size_t) -{ - return value; -} - -template -void HashedDictionary::isInAttrImpl(const AttrType & attr, const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const -{ - const auto null_value = std::get(hierarchical_attribute->null_values); - const auto rows = out.size(); - - for (const auto row : ext::range(0, rows)) - { - auto id = getAt(child_ids, row); - const auto ancestor_id = getAt(ancestor_ids, row); - - for (size_t i = 0; id != null_value && id != ancestor_id && i < DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH; ++i) - { - auto it = attr.find(id); - if (it != std::end(attr)) - id = second(*it); - else - break; - } - - out[row] = id != null_value && id == ancestor_id; - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} -template -void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const -{ - if (!sparse) - return isInAttrImpl(*std::get>(hierarchical_attribute->maps), child_ids, ancestor_ids, out); - return isInAttrImpl(*std::get>(hierarchical_attribute->sparse_maps), child_ids, ancestor_ids, out); -} - -void HashedDictionary::isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_ids, out); -} - -void HashedDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const -{ - isInImpl(child_ids, ancestor_id, out); -} - -void HashedDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const -{ - isInImpl(child_id, ancestor_ids, out); -} - -ColumnPtr HashedDictionary::getColumn( +template +ColumnPtr HashedDictionary::getColumn( const std::string & attribute_name, const DataTypePtr & result_type, const Columns & key_columns, - const DataTypes &, + const DataTypes & key_types, const ColumnPtr & default_values_column) const { - ColumnPtr result; + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + dict_struct.validateKeyTypes(key_types); - PaddedPODArray backup_storage; - const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage); + Arena temporary_complex_key_arena; - auto size = ids.size(); + const DictionaryAttribute & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + DefaultValueProvider default_value_provider(dictionary_attribute.null_value, default_values_column); - const auto & attribute = getAttribute(attribute_name); - const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + const auto & requested_keys = extractor.getKeys(); - auto type_call = [&](const auto & dictionary_attribute_type) + auto result_column = dictionary_attribute.type->createColumn(); + result_column->reserve(requested_keys.size()); + + size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + const auto & attribute = attributes[attribute_index]; + + Field row_value_to_insert; + + if (unlikely(attribute.is_complex_type)) { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - using ColumnProvider = DictionaryAttributeColumnProvider; + auto & attribute_container = std::get>(attribute.container); - const auto attribute_null_value = std::get(attribute.null_values); - AttributeType null_value = static_cast(attribute_null_value); - DictionaryDefaultValueExtractor default_value_extractor(std::move(null_value), default_values_column); - - auto column = ColumnProvider::getColumn(dictionary_attribute, size); - - if constexpr (std::is_same_v) + for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) { - auto * out = column.get(); + auto & requested_key = requested_keys[requested_key_index]; + auto it = attribute_container.find(requested_key); - getItemsImpl( - attribute, - ids, - [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, - default_value_extractor); + if (it != attribute_container.end()) + row_value_to_insert = it->second; + else + row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); + + result_column->insert(row_value_to_insert); } - else - { - auto & out = column->getData(); - - getItemsImpl( - attribute, - ids, - [&](const size_t row, const auto value) { return out[row] = value; }, - default_value_extractor); - } - - result = std::move(column); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - if (attribute.nullable_set) + } + else { - ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(size, false); - ColumnUInt8::Container& vec_null_map_to = col_null_map_to->getData(); - - for (size_t row = 0; row < ids.size(); ++row) + auto type_call = [&](const auto & dictionary_attribute_type) { - auto id = ids[row]; + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnType = std::conditional_t< + std::is_same_v, + ColumnString, + std::conditional_t, ColumnDecimal, ColumnVector>>; - if (attribute.nullable_set->find(id) != nullptr) - vec_null_map_to[row] = true; - } + auto & attribute_container = std::get>(attribute.container); + ColumnType & result_column_typed = static_cast(*result_column); - result = ColumnNullable::create(result, std::move(col_null_map_to)); + if constexpr (std::is_same_v) + { + for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) + { + auto & requested_key = requested_keys[requested_key_index]; + auto it = attribute_container.find(requested_key); + + if (it != attribute_container.end()) + { + auto item = it->second; + result_column->insertData(item.data, item.size); + } + else + { + row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); + result_column->insert(row_value_to_insert); + } + } + } + else + { + auto & result_data = result_column_typed.getData(); + + for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) + { + auto & requested_key = requested_keys[requested_key_index]; + auto it = attribute_container.find(requested_key); + + if (it != attribute_container.end()) + { + auto item = it->second; + result_data.emplace_back(item); + } + else + { + row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); + result_data.emplace_back(row_value_to_insert.get>()); + } + } + } + }; + + callOnDictionaryAttributeType(attribute.type, type_call); } - return result; + query_count.fetch_add(requested_keys.size(), std::memory_order_relaxed); + + return result_column; } -ColumnUInt8::Ptr HashedDictionary::hasKeys(const Columns & key_columns, const DataTypes &) const +template +ColumnUInt8::Ptr HashedDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const { - PaddedPODArray backup_storage; - const auto& ids = getColumnVectorData(this, key_columns.front(), backup_storage); + if (dictionary_key_type == DictionaryKeyType::complex) + dict_struct.validateKeyTypes(key_types); - size_t ids_count = ext::size(ids); + Arena complex_keys_arena; + DictionaryKeysExtractor extractor(key_columns, complex_keys_arena); - auto result = ColumnUInt8::create(ext::size(ids)); + const auto & keys = extractor.getKeys(); + size_t keys_size = keys.size(); + + auto result = ColumnUInt8::create(keys_size); auto& out = result->getData(); const auto & attribute = attributes.front(); @@ -220,52 +172,157 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Columns & key_columns, const Da { using Type = std::decay_t; using AttributeType = typename Type::AttributeType; - has(attribute, ids, out); + using ValueType = DictionaryValueType; + + const auto & attribute_map = std::get>(attribute.container); + + for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) + { + const auto & requested_key = keys[requested_key_index]; + out[requested_key_index] = attribute_map.find(requested_key) != attribute_map.end(); + } }; callOnDictionaryAttributeType(attribute.type, type_call); - query_count.fetch_add(ids_count, std::memory_order_relaxed); + query_count.fetch_add(keys_size, std::memory_order_relaxed); return result; } -void HashedDictionary::createAttributes() +template +ColumnPtr HashedDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const UInt64 null_value = dictionary_attribute.null_value.get(); + const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) + { + return parent_keys_map.find(key) != parent_keys_map.end(); + }; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = parent_keys_map.find(hierarchy_key); + + if (it == parent_keys_map.end()) + return result; + + result = it->second; + + return result; + }; + + auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, null_value, is_key_valid_func, get_parent_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + + return dictionary_hierarchy_array; + } + else + return nullptr; +} + +template +ColumnUInt8::Ptr HashedDictionary::isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + PaddedPODArray keys_in_backup_storage; + const auto & keys_in = getColumnVectorData(this, in_key_column, keys_in_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const UInt64 null_value = dictionary_attribute.null_value.get(); + const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) + { + return parent_keys_map.find(key) != parent_keys_map.end(); + }; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = parent_keys_map.find(hierarchy_key); + + if (it == parent_keys_map.end()) + return result; + + result = it->second; + + return result; + }; + + auto is_in_hierarchy_result = isInKeysHierarchy(keys, keys_in, null_value, is_key_valid_func, get_parent_func); + + auto result = ColumnUInt8::create(); + result->getData() = std::move(is_in_hierarchy_result); + + return result; + } + else + return nullptr; +} + +template +void HashedDictionary::createAttributes() { const auto size = dict_struct.attributes.size(); attributes.reserve(size); - for (const auto & attribute : dict_struct.attributes) + for (const auto & dictionary_attribute : dict_struct.attributes) { - attribute_index_by_name.emplace(attribute.name, attributes.size()); - attributes.push_back(createAttribute(attribute, attribute.null_value)); + bool is_complex_type = dictionary_attribute.is_nullable || dictionary_attribute.is_array; - if (attribute.hierarchical) + auto type_call = [&, this](const auto & dictionary_attribute_type) { - hierarchical_attribute = &attributes.back(); + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; - if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64) - throw Exception{full_name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; - } + std::unique_ptr string_arena = std::is_same_v ? std::make_unique() : nullptr; + + if (is_complex_type) + { + Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, CollectionType(), std::move(string_arena)}; + attributes.emplace_back(std::move(attribute)); + } + else + { + Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, CollectionType(), std::move(string_arena)}; + attributes.emplace_back(std::move(attribute)); + } + }; + + callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); } } -void HashedDictionary::blockToAttributes(const Block & block) -{ - const auto & id_column = *block.safeGetByPosition(0).column; - - for (const size_t attribute_idx : ext::range(0, attributes.size())) - { - const IColumn & attribute_column = *block.safeGetByPosition(attribute_idx + 1).column; - auto & attribute = attributes[attribute_idx]; - - for (const auto row_idx : ext::range(0, id_column.size())) - if (setAttributeValue(attribute, id_column[row_idx].get(), attribute_column[row_idx])) - ++element_count; - } -} - -void HashedDictionary::updateData() +template +void HashedDictionary::updateData() { if (!saved_block || saved_block->rows() == 0) { @@ -288,34 +345,50 @@ void HashedDictionary::updateData() } else { + Arena temporary_complex_key_arena; + + size_t skip_keys_size_offset = dict_struct.getKeysSize(); + + Columns saved_block_key_columns; + saved_block_key_columns.reserve(skip_keys_size_offset); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + saved_block_key_columns.emplace_back(saved_block->safeGetByPosition(i).column); + + DictionaryKeysExtractor saved_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); + const auto & saved_keys_extracted_from_block = saved_keys_extractor.getKeys(); + auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); while (Block block = stream->read()) { - const auto & saved_id_column = *saved_block->safeGetByPosition(0).column; - const auto & update_id_column = *block.safeGetByPosition(0).column; + /// TODO: Rewrite + Columns block_key_columns; + block_key_columns.reserve(skip_keys_size_offset); - std::unordered_map> update_ids; - for (size_t row = 0; row < update_id_column.size(); ++row) + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + block_key_columns.emplace_back(block.safeGetByPosition(i).column); + + DictionaryKeysExtractor block_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); + const auto & keys_extracted_from_block = block_keys_extractor.getKeys(); + + absl::flat_hash_map, DefaultHash> update_keys; + for (size_t row = 0; row < keys_extracted_from_block.size(); ++row) { - const auto id = update_id_column.get64(row); - update_ids[id].push_back(row); + const auto key = keys_extracted_from_block[row]; + update_keys[key].push_back(row); } - const size_t saved_rows = saved_id_column.size(); - IColumn::Filter filter(saved_rows); - std::unordered_map>::iterator it; + IColumn::Filter filter(saved_keys_extracted_from_block.size()); - for (size_t row = 0; row < saved_id_column.size(); ++row) + for (size_t row = 0; row < saved_keys_extracted_from_block.size(); ++row) { - auto id = saved_id_column.get64(row); - it = update_ids.find(id); - - if (it != update_ids.end()) - filter[row] = 0; - else - filter[row] = 1; + auto key = saved_keys_extracted_from_block[row]; + auto it = update_keys.find(key); + filter[row] = (it == update_keys.end()); } auto block_columns = block.mutateColumns(); @@ -323,12 +396,12 @@ void HashedDictionary::updateData() { auto & column = saved_block->safeGetByPosition(attribute_idx).column; const auto & filtered_column = column->filter(filter, -1); - block_columns[attribute_idx]->insertRangeFrom(*filtered_column.get(), 0, filtered_column->size()); } saved_block->setColumns(std::move(block_columns)); } + stream->readSuffix(); } @@ -339,48 +412,106 @@ void HashedDictionary::updateData() } } -template -void HashedDictionary::resize(Attribute & attribute, size_t added_rows) +template +void HashedDictionary::blockToAttributes(const Block & block [[maybe_unused]]) { - if (!sparse) + Arena temporary_complex_key_arena; + + size_t skip_keys_size_offset = dict_struct.getKeysSize(); + + Columns key_columns; + key_columns.reserve(skip_keys_size_offset); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + key_columns.emplace_back(block.safeGetByPosition(i).column); + + DictionaryKeysExtractor keys_extractor(key_columns, temporary_complex_key_arena); + const auto & keys_extracted_from_block = keys_extractor.getKeys(); + + Field column_value_to_insert; + + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) { - const auto & map_ref = std::get>(attribute.maps); - added_rows += map_ref->size(); - map_ref->reserve(added_rows); - } - else - { - const auto & map_ref = std::get>(attribute.sparse_maps); - added_rows += map_ref->size(); - map_ref->resize(added_rows); + const IColumn & attribute_column = *block.safeGetByPosition(skip_keys_size_offset + attribute_index).column; + auto & attribute = attributes[attribute_index]; + + getAttributeContainer(attribute_index, [&](auto & container) + { + using ContainerType = std::decay_t; + using AttributeValueType = typename ContainerType::mapped_type; + + for (size_t key_index = 0; key_index < keys_extracted_from_block.size(); ++key_index) + { + auto key = keys_extracted_from_block[key_index]; + auto it = container.find(key); + + if (it != container.end()) + continue; + + if constexpr (std::is_same_v) + key = copyKeyInArena(key); + + attribute_column.get(key_index, column_value_to_insert); + + if constexpr (std::is_same_v) + { + container.insert({key, column_value_to_insert}); + } + else if constexpr (std::is_same_v) + { + String & value_to_insert = column_value_to_insert.get(); + size_t value_to_insert_size = value_to_insert.size(); + + const char * string_in_arena = attribute.string_arena->insert(value_to_insert.data(), value_to_insert_size); + + StringRef string_in_arena_reference = StringRef{string_in_arena, value_to_insert_size}; + container.insert({key, string_in_arena_reference}); + } + else + { + auto value_to_insert = column_value_to_insert.get>(); + container.insert({key, value_to_insert}); + } + + ++element_count; + } + }); } } -template <> -void HashedDictionary::resize(Attribute & attribute, size_t added_rows) +template +void HashedDictionary::resize(size_t added_rows) { - resize(attribute, added_rows); -} - -void HashedDictionary::resize(size_t added_rows) -{ - if (!added_rows) + if (unlikely(!added_rows)) return; - for (auto & attribute : attributes) + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) { - auto type_call = [&](const auto & dictionary_attribute_type) + getAttributeContainer(attribute_index, [added_rows](auto & attribute_map) { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - resize(attribute, added_rows); - }; + size_t reserve_size = added_rows + attribute_map.size(); - callOnDictionaryAttributeType(attribute.type, type_call); + if constexpr (sparse) + attribute_map.resize(reserve_size); + else + attribute_map.reserve(reserve_size); + }); } } -void HashedDictionary::loadData() +template +StringRef HashedDictionary::copyKeyInArena(StringRef key) +{ + size_t key_size = key.size; + char * place_for_key = complex_key_arena.alloc(key_size); + memcpy(reinterpret_cast(place_for_key), reinterpret_cast(key.data), key_size); + StringRef updated_key{place_for_key, key_size}; + return updated_key; +} + +template +void HashedDictionary::loadData() { if (!source_ptr->hasUpdateField()) { @@ -400,262 +531,94 @@ void HashedDictionary::loadData() updateData(); if (require_nonempty && 0 == element_count) - throw Exception{full_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY}; + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, + "({}): dictionary source is empty and 'require_nonempty' property is set.", + full_name); } -template -void HashedDictionary::addAttributeSize(const Attribute & attribute) -{ - if (!sparse) - { - const auto & map_ref = std::get>(attribute.maps); - bytes_allocated += sizeof(CollectionType) + map_ref->getBufferSizeInBytes(); - bucket_count = map_ref->getBufferSizeInCells(); - } - else - { - const auto & map_ref = std::get>(attribute.sparse_maps); - bucket_count = map_ref->bucket_count(); - - /** TODO: more accurate calculation */ - bytes_allocated += sizeof(SparseCollectionType); - bytes_allocated += bucket_count; - bytes_allocated += map_ref->size() * (sizeof(Key) + sizeof(T)); - } -} - -template <> -void HashedDictionary::addAttributeSize(const Attribute & attribute) -{ - addAttributeSize(attribute); - bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); -} - -void HashedDictionary::calculateBytesAllocated() +template +void HashedDictionary::calculateBytesAllocated() { bytes_allocated += attributes.size() * sizeof(attributes.front()); - for (const auto & attribute : attributes) + for (size_t i = 0; i < attributes.size(); ++i) + { + getAttributeContainer(i, [&](const auto & container) + { + /// TODO: Calculate + bytes_allocated += sizeof(container); + }); + } + + bytes_allocated += complex_key_arena.size(); +} + +template +BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +{ + PaddedPODArray keys; + + if (!attributes.empty()) + getAttributeContainer(0, [&](auto & container) + { + keys.reserve(container.size()); + + for (const auto & [key, value] : container) + { + (void)(value); + keys.emplace_back(key); + } + }); + + if constexpr (dictionary_key_type == DictionaryKeyType::simple) + return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); + else + return std::make_shared(shared_from_this(), max_block_size, keys, column_names); +} + +template +template +void HashedDictionary::getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) +{ + assert(attribute_index < attributes.size()); + + auto & attribute = attributes[attribute_index]; + + if (unlikely(attribute.is_complex_type)) + { + auto & attribute_container = std::get>(attribute.container); + std::forward(get_container_func)(attribute_container); + } + else { auto type_call = [&](const auto & dictionary_attribute_type) { using Type = std::decay_t; using AttributeType = typename Type::AttributeType; - addAttributeSize(attribute); + using ValueType = DictionaryValueType; + + auto & attribute_container = std::get>(attribute.container); + std::forward(get_container_func)(attribute_container); }; callOnDictionaryAttributeType(attribute.type, type_call); } } -template -void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) +template +template +void HashedDictionary::getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const { - attribute.null_values = T(null_value.get()); - if (!sparse) - attribute.maps = std::make_unique>(); - else - attribute.sparse_maps = std::make_unique>(); -} - -template <> -void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) -{ - attribute.string_arena = std::make_unique(); - const String & string = null_value.get(); - const char * string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - attribute.null_values.emplace(string_in_arena, string.size()); - - if (!sparse) - attribute.maps = std::make_unique>(); - else - attribute.sparse_maps = std::make_unique>(); -} - -HashedDictionary::Attribute HashedDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value) -{ - auto nullable_set = attribute.is_nullable ? std::make_optional() : std::optional{}; - Attribute attr{attribute.underlying_type, std::move(nullable_set), {}, {}, {}, {}}; - - auto type_call = [&, this](const auto &dictionary_attribute_type) + const_cast *>(this)->getAttributeContainer(attribute_index, [&](auto & attribute_container) { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - createAttributeImpl(attr, null_value); - }; - - callOnDictionaryAttributeType(attribute.underlying_type, type_call); - - return attr; + std::forward(get_container_func)(attribute_container); + }); } - -template -void HashedDictionary::getItemsAttrImpl( - const MapType & attr, - const PaddedPODArray & ids, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const -{ - const auto rows = ext::size(ids); - - for (const auto i : ext::range(0, rows)) - { - const auto it = attr.find(ids[i]); - set_value(i, it != attr.end() ? static_cast(second(*it)) : default_value_extractor[i]); - } - - query_count.fetch_add(rows, std::memory_order_relaxed); -} - -template -void HashedDictionary::getItemsImpl( - const Attribute & attribute, - const PaddedPODArray & ids, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const -{ - if (!sparse) - return getItemsAttrImpl(*std::get>(attribute.maps), ids, set_value, default_value_extractor); - return getItemsAttrImpl(*std::get>(attribute.sparse_maps), ids, set_value, default_value_extractor); -} - - -template -bool HashedDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T value) -{ - if (!sparse) - { - auto & map = *std::get>(attribute.maps); - return map.insert({id, value}).second; - } - else - { - auto & map = *std::get>(attribute.sparse_maps); - return map.insert({id, value}).second; - } -} - -template <> -bool HashedDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const String value) -{ - const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size()); - return setAttributeValueImpl(attribute, id, StringRef{string_in_arena, value.size()}); -} - -bool HashedDictionary::setAttributeValue(Attribute & attribute, const Key id, const Field & value) -{ - bool result = false; - - auto type_call = [&, this](const auto &dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - - if (attribute.nullable_set) - { - if (value.isNull()) - { - result = attribute.nullable_set->insert(id).second; - return; - } - else - { - attribute.nullable_set->erase(id); - } - } - - result = setAttributeValueImpl(attribute, id, value.get()); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - return result; -} - -const HashedDictionary::Attribute & HashedDictionary::getAttribute(const std::string & attribute_name) const -{ - const auto it = attribute_index_by_name.find(attribute_name); - if (it == std::end(attribute_index_by_name)) - throw Exception{full_name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; - - return attributes[it->second]; -} - -template -void HashedDictionary::has(const Attribute & attribute, const PaddedPODArray & ids, PaddedPODArray & out) const -{ - const auto & attr = *std::get>(attribute.maps); - const auto rows = ext::size(ids); - - for (const auto i : ext::range(0, rows)) - { - out[i] = attr.find(ids[i]) != nullptr; - - if (attribute.nullable_set && !out[i]) - out[i] = attribute.nullable_set->find(ids[i]) != nullptr; - } -} - -template <> -void HashedDictionary::has(const Attribute & attribute, const PaddedPODArray & ids, PaddedPODArray & out) const -{ - has(attribute, ids, out); -} - -template -PaddedPODArray HashedDictionary::getIdsAttrImpl(const AttrType & attr) const -{ - PaddedPODArray ids; - ids.reserve(attr.size()); - for (const auto & value : attr) - ids.push_back(first(value)); - - return ids; -} -template -PaddedPODArray HashedDictionary::getIds(const Attribute & attribute) const -{ - if (!sparse) - return getIdsAttrImpl(*std::get>(attribute.maps)); - return getIdsAttrImpl(*std::get>(attribute.sparse_maps)); -} - -template <> -PaddedPODArray HashedDictionary::getIds(const Attribute & attribute) const -{ - return getIds(attribute); -} - -PaddedPODArray HashedDictionary::getIds() const -{ - const auto & attribute = attributes.front(); - PaddedPODArray result; - - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - /// TODO: Check if order is satisfied - result = getIds(attribute); - - if (attribute.nullable_set) - { - for (const auto& value: *attribute.nullable_set) - result.push_back(value.getKey()); - } - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - - return result; -} - -BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const -{ - using BlockInputStreamType = DictionaryBlockInputStream; - return std::make_shared(shared_from_this(), max_block_size, getIds(), column_names); -} +template class HashedDictionary; +template class HashedDictionary; +template class HashedDictionary; +template class HashedDictionary; void registerDictionaryHashed(DictionaryFactory & factory) { @@ -664,10 +627,13 @@ void registerDictionaryHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, + DictionaryKeyType dictionary_key_type, bool sparse) -> DictionaryPtr { - if (dict_struct.key) - throw Exception{"'key' is not supported for dictionary of layout 'hashed'", ErrorCodes::UNSUPPORTED_METHOD}; + if (dictionary_key_type == DictionaryKeyType::simple && dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary"); + else if (dictionary_key_type == DictionaryKeyType::complex && dict_struct.id) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary"); if (dict_struct.range_min || dict_struct.range_max) throw Exception{full_name @@ -678,13 +644,34 @@ void registerDictionaryHashed(DictionaryFactory & factory) const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse); + + if (dictionary_key_type == DictionaryKeyType::simple) + { + if (sparse) + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + else + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + } + else + { + if (sparse) + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + else + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + } }; + using namespace std::placeholders; + factory.registerLayout("hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), /* sparse = */ false); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false); factory.registerLayout("sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), /* sparse = */ true); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); + factory.registerLayout("complex_key_hashed", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); + factory.registerLayout("complex_key_sparse_hashed", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); + } } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index ab37f1528ca..a3290c800d1 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -4,17 +4,22 @@ #include #include #include -#include -#include -#include + +#include +#include +#include + #include #include -#include -#include -#include "DictionaryStructure.h" -#include "IDictionary.h" -#include "IDictionarySource.h" -#include "DictionaryHelpers.h" +#include + +#include +#include + +#include +#include +#include +#include /** This dictionary stores all content in a hash table in memory * (a separate Key -> Value map for each attribute) @@ -24,19 +29,32 @@ namespace DB { +template class HashedDictionary final : public IDictionary { public: + using KeyType = std::conditional_t; + static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary"); + HashedDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, - bool sparse_, BlockPtr saved_block_ = nullptr); - std::string getTypeName() const override { return sparse ? "SparseHashed" : "Hashed"; } + std::string getTypeName() const override + { + if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse) + return "SparseHashed"; + else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse) + return "Hashed"; + else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse) + return "ComplexKeySpareseHashed"; + else + return "ComplexKeyHashed"; + } size_t getBytesAllocated() const override { return bytes_allocated; } @@ -50,7 +68,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block); + return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -61,14 +79,10 @@ public: bool isInjective(const std::string & attribute_name) const override { - return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; + return dict_struct.getAttribute(attribute_name).injective; } - bool hasHierarchy() const override { return hierarchical_attribute; } - - void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; - - DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; } + DictionaryKeyType getKeyType() const override { return dictionary_key_type; } ColumnPtr getColumn( const std::string& attribute_name, @@ -79,88 +93,57 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - void isInVectorVector( - const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; - void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; - void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } + + ColumnPtr getHierarchy(ColumnPtr hierarchy_attribute_column, const DataTypePtr & hierarchy_attribute_type) const override; + + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: template - using CollectionType = HashMap; - template - using CollectionPtrType = std::unique_ptr>; + using CollectionTypeNonSparse = std::conditional_t, absl::flat_hash_map>>; #if !defined(ARCADIA_BUILD) - template - using SparseCollectionType = google::sparse_hash_map>; + template + using SparseHashMap = google::sparse_hash_map>; #else template - using SparseCollectionType = google::sparsehash::sparse_hash_map>; + using SparseHashMap = google::sparsehash::sparse_hash_map>; #endif template - using SparseCollectionPtrType = std::unique_ptr>; + using CollectionTypeSparse = std::conditional_t, SparseHashMap>; - using NullableSet = HashSet>; + template + using CollectionType = std::conditional_t, CollectionTypeNonSparse>; struct Attribute final { AttributeUnderlyingType type; - std::optional nullable_set; - + bool is_complex_type; std::variant< - UInt8, - UInt16, - UInt32, - UInt64, - UInt128, - Int8, - Int16, - Int32, - Int64, - Decimal32, - Decimal64, - Decimal128, - Float32, - Float64, - StringRef> - null_values; - std::variant< - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType, - CollectionPtrType> - maps; - std::variant< - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType, - SparseCollectionPtrType> - sparse_maps; + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType, + CollectionType> + container; std::unique_ptr string_arena; }; @@ -172,76 +155,39 @@ private: void loadData(); - template - void addAttributeSize(const Attribute & attribute); - void calculateBytesAllocated(); - template - void createAttributeImpl(Attribute & attribute, const Field & null_value); + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); - Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value); + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const; - template - void getItemsAttrImpl( - const MapType & attr, - const PaddedPODArray & ids, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const; - - template - void getItemsImpl( - const Attribute & attribute, - const PaddedPODArray & ids, - ValueSetter && set_value, - DefaultValueExtractor & default_value_extractor) const; - - template - bool setAttributeValueImpl(Attribute & attribute, const Key id, const T value); - - bool setAttributeValue(Attribute & attribute, const Key id, const Field & value); - - const Attribute & getAttribute(const std::string & attribute_name) const; - - template - void has(const Attribute & attribute, const PaddedPODArray & ids, PaddedPODArray & out) const; - - template - PaddedPODArray getIdsAttrImpl(const AttrType & attr) const; - template - PaddedPODArray getIds(const Attribute & attribute) const; - - PaddedPODArray getIds() const; - - /// Preallocates the hashtable based on query progress - /// (Only while loading all data). - /// - /// @see preallocate - template - void resize(Attribute & attribute, size_t added_rows); void resize(size_t added_rows); - template - void isInAttrImpl(const AttrType & attr, const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; - template - void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; + StringRef copyKeyInArena(StringRef key); const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; const bool require_nonempty; - const bool sparse; - std::map attribute_index_by_name; std::vector attributes; - const Attribute * hierarchical_attribute = nullptr; size_t bytes_allocated = 0; size_t element_count = 0; size_t bucket_count = 0; mutable std::atomic query_count{0}; + /// TODO: Remove BlockPtr saved_block; + Arena complex_key_arena; }; +extern template class HashedDictionary; +extern template class HashedDictionary; + +extern template class HashedDictionary; +extern template class HashedDictionary; + } diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp new file mode 100644 index 00000000000..c7e22419fc0 --- /dev/null +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -0,0 +1,150 @@ +#include "HierarchyDictionariesUtils.h" + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +namespace +{ + HashMap getHierarchyMapImpl(const IDictionary * dictionary, const DictionaryAttribute & dictionary_attribute, const PaddedPODArray & initial_keys_to_request, const DataTypePtr & key_type) + { + UInt64 null_value = dictionary_attribute.null_value.get(); + + ColumnPtr key_to_request_column = ColumnVector::create(); + auto * key_to_request_column_typed = static_cast *>(key_to_request_column->assumeMutable().get()); + + UInt64 key_not_in_storage_value = std::numeric_limits::max(); + ColumnPtr key_not_in_storage_default_value_column = ColumnVector::create(initial_keys_to_request.size(), key_not_in_storage_value); + + PaddedPODArray & keys_to_request = key_to_request_column_typed->getData(); + keys_to_request.assign(initial_keys_to_request); + + PaddedPODArray next_keys_to_request; + HashSet already_requested_keys; + + HashMap key_to_parent_key; + + while (!keys_to_request.empty()) + { + key_to_parent_key.reserve(key_to_parent_key.size() + keys_to_request.size()); + + auto parent_key_column + = dictionary->getColumn(dictionary_attribute.name, dictionary_attribute.type, {key_to_request_column}, {key_type}, key_not_in_storage_default_value_column); + + const auto * parent_key_column_typed = checkAndGetColumn>(*parent_key_column); + if (!parent_key_column_typed) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Parent key column should be UInt64"); + + const auto & parent_keys = parent_key_column_typed->getData(); + next_keys_to_request.clear(); + + for (size_t i = 0; i < keys_to_request.size(); ++i) + { + auto key = keys_to_request[i]; + auto parent_key = parent_keys[i]; + + if (parent_key == key_not_in_storage_value) + continue; + + key_to_parent_key[key] = parent_key; + + if (parent_key == null_value || + already_requested_keys.find(parent_key) != nullptr) + continue; + + already_requested_keys.insert(parent_key); + next_keys_to_request.emplace_back(parent_key); + } + + keys_to_request.clear(); + keys_to_request.assign(next_keys_to_request); + } + + return key_to_parent_key; + } +} + +ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type) +{ + const auto * key_column_typed = checkAndGetColumn>(*key_column); + if (!key_column_typed) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); + + const auto & dictionary_structure = dictionary->getStructure(); + const auto & dictionary_attribute = dictionary_structure.attributes[0]; + + const PaddedPODArray & requested_keys = key_column_typed->getData(); + HashMap key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + + auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; + + auto get_parent_key_func = [&](auto & key) + { + std::optional result; + + auto it = key_to_parent_key.find(key); + + if (it != nullptr) + result = it->getMapped(); + + return result; + }; + + UInt64 null_value = dictionary_attribute.null_value.get(); + + auto dictionary_hierarchy_array = getKeysHierarchyArray(requested_keys, null_value, is_key_valid_func, get_parent_key_func); + return dictionary_hierarchy_array; +} + +ColumnUInt8::Ptr isInHierarchyDefaultImplementation( + const IDictionary * dictionary, + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) +{ + const auto * key_column_typed = checkAndGetColumn>(*key_column); + if (!key_column_typed) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); + + const auto * in_key_column_typed = checkAndGetColumn>(*in_key_column); + if (!in_key_column_typed) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); + + const auto & dictionary_structure = dictionary->getStructure(); + const auto & dictionary_attribute = dictionary_structure.attributes[0]; + + const PaddedPODArray & requested_keys = key_column_typed->getData(); + HashMap key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + + auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; + + auto get_parent_key_func = [&](auto & key) + { + std::optional result; + + auto it = key_to_parent_key.find(key); + + if (it != nullptr) + result = it->getMapped(); + + return result; + }; + + UInt64 null_value = dictionary_attribute.null_value.get(); + const auto & in_keys = in_key_column_typed->getData(); + + auto is_in_hierarchy_result = isInKeysHierarchy(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func); + + auto result = ColumnUInt8::create(); + result->getData() = std::move(is_in_hierarchy_result); + + return result; +} + +} diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h new file mode 100644 index 00000000000..ba2a74db298 --- /dev/null +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -0,0 +1,197 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +template +struct ElementsAndOffsets +{ + PaddedPODArray elements; + PaddedPODArray offsets; +}; + +template +struct IsKeyValidFuncInterface +{ + bool operator()(T key [[maybe_unused]]) { return false; } +}; + +template +struct GetParentKeyFuncInterface +{ + std::optional operator()(T key [[maybe_unused]]) { return {}; } +}; + +template +ElementsAndOffsets getKeysHierarchy( + const PaddedPODArray & hierarchy_keys, + const KeyType & hierarchy_null_value, + IsKeyValidFunc && is_key_valid_func, + GetParentKeyFunc && get_parent_func) +{ + size_t hierarchy_keys_size = hierarchy_keys.size(); + + PaddedPODArray elements; + elements.reserve(hierarchy_keys_size); + + PaddedPODArray offsets; + offsets.reserve(hierarchy_keys_size); + + struct OffsetInArray + { + size_t offset_index; + size_t array_element_offset; + }; + + HashMap already_processes_keys_to_offset; + already_processes_keys_to_offset.reserve(hierarchy_keys_size); + + for (size_t i = 0; i < hierarchy_keys_size; ++i) + { + auto hierarchy_key = hierarchy_keys[i]; + size_t current_hierarchy_depth = 0; + + bool is_key_valid = std::forward(is_key_valid_func)(hierarchy_key); + + if (!is_key_valid) + { + offsets.emplace_back(elements.size()); + continue; + } + + while (true) + { + const auto * it = already_processes_keys_to_offset.find(hierarchy_key); + + if (it) + { + const auto & index = it->getMapped(); + + size_t offset = index.offset_index; + + bool is_loop = (offset == offsets.size()); + + if (unlikely(is_loop)) + break; + + size_t array_element_offset = index.array_element_offset; + + size_t previous_offset_size = offset > 0 ? offsets[offset - 1] : 0; + size_t start_index = previous_offset_size + array_element_offset; + size_t end_index = offsets[offset]; + + current_hierarchy_depth += end_index - start_index; + + /// TODO: Insert part of pod array into itself + while (start_index < end_index) + { + elements.emplace_back(elements[start_index]); + ++start_index; + } + + break; + } + + if (hierarchy_key == hierarchy_null_value || current_hierarchy_depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH) + break; + + already_processes_keys_to_offset[hierarchy_key] = {offsets.size(), current_hierarchy_depth}; + elements.emplace_back(hierarchy_key); + ++current_hierarchy_depth; + + std::optional parent_key = std::forward(get_parent_func)(hierarchy_key); + + if (!parent_key.has_value()) + break; + + hierarchy_key = *parent_key; + } + + offsets.emplace_back(elements.size()); + } + + ElementsAndOffsets result = {std::move(elements), std::move(offsets)}; + + return result; +} + +template +ColumnPtr getKeysHierarchyArray( + const PaddedPODArray & hierarchy_keys, + const KeyType & hierarchy_null_value, + IsKeyValidFunc && is_key_valid_func, + GetParentKeyFunc && get_parent_func) +{ + auto elements_and_offsets = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); + + auto elements_column = ColumnVector::create(); + elements_column->getData() = std::move(elements_and_offsets.elements); + + auto offsets_column = ColumnVector::create(); + offsets_column->getData() = std::move(elements_and_offsets.offsets); + + auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); + return column_array; +} + +template +PaddedPODArray isInKeysHierarchy( + const PaddedPODArray & hierarchy_keys, + const PaddedPODArray & hierarchy_in_keys, + const KeyType & hierarchy_null_value, + IsKeyValidFunc && is_key_valid_func, + GetParentKeyFunc && get_parent_func) +{ + assert(hierarchy_keys.size() == hierarchy_in_keys.size()); + + PaddedPODArray result; + result.resize_fill(hierarchy_keys.size()); + + ElementsAndOffsets hierarchy = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); + + auto & offsets = hierarchy.offsets; + auto & elements = hierarchy.elements; + + for (size_t i = 0; i < offsets.size(); ++i) + { + size_t i_elements_start = i > 0 ? offsets[i - 1] : 0; + size_t i_elements_end = offsets[i]; + + auto & key_to_find = hierarchy_in_keys[i]; + + const auto * begin = elements.begin() + i_elements_start; + const auto * end = elements.begin() + i_elements_end; + + const auto * it = std::find(begin, end, key_to_find); + + bool contains_key = (it != end); + result[i] = contains_key; + } + + return result; +} + +ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type); + +ColumnUInt8::Ptr isInHierarchyDefaultImplementation( + const IDictionary * dictionary, + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type); + +} diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 4d51747a652..8b10cd7a819 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -24,8 +24,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -struct IDictionaryBase; -using DictionaryPtr = std::unique_ptr; +struct IDictionary; +using DictionaryPtr = std::unique_ptr; /** DictionaryKeyType provides IDictionary client information about * which key type is supported by dictionary. @@ -47,13 +47,11 @@ enum class DictionaryKeyType /** * Base class for Dictionaries implementation. */ -struct IDictionaryBase : public IExternalLoadable +struct IDictionary : public IExternalLoadable { - using Key = UInt64; - - IDictionaryBase(const StorageID & dict_id_) - : dict_id(dict_id_) - , full_name(dict_id.getInternalDictionaryName()) + explicit IDictionary(const StorageID & dictionary_id_) + : dictionary_id(dictionary_id_) + , full_name(dictionary_id.getInternalDictionaryName()) { } @@ -61,14 +59,14 @@ struct IDictionaryBase : public IExternalLoadable StorageID getDictionaryID() const { std::lock_guard lock{name_mutex}; - return dict_id; + return dictionary_id; } void updateDictionaryName(const StorageID & new_name) const { std::lock_guard lock{name_mutex}; - assert(new_name.uuid == dict_id.uuid && dict_id.uuid != UUIDHelpers::Nil); - dict_id = new_name; + assert(new_name.uuid == dictionary_id.uuid && dictionary_id.uuid != UUIDHelpers::Nil); + dictionary_id = new_name; } const std::string & getLoadableName() const override final { return getFullName(); } @@ -80,8 +78,9 @@ struct IDictionaryBase : public IExternalLoadable std::string getDatabaseOrNoDatabaseTag() const { - if (!dict_id.database_name.empty()) - return dict_id.database_name; + if (!dictionary_id.database_name.empty()) + return dictionary_id.database_name; + return NO_DATABASE_TAG; } @@ -159,74 +158,55 @@ struct IDictionaryBase : public IExternalLoadable const Columns & key_columns, const DataTypes & key_types) const = 0; + virtual bool hasHierarchy() const { return false; } + + virtual ColumnPtr getHierarchy( + ColumnPtr key_column [[maybe_unused]], + const DataTypePtr & key_type [[maybe_unused]]) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Hierarchy is not supported for {} dictionary.", + getDictionaryID().getNameForLogs()); + } + + virtual ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column [[maybe_unused]], + ColumnPtr in_key_column [[maybe_unused]], + const DataTypePtr & key_type [[maybe_unused]]) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Hierarchy is not supported for {} dictionary.", + getDictionaryID().getNameForLogs()); + } + virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0; bool supportUpdates() const override { return true; } bool isModified() const override { - auto source = getSource(); + const auto * source = getSource(); return source && source->isModified(); } virtual std::exception_ptr getLastException() const { return {}; } - std::shared_ptr shared_from_this() + std::shared_ptr shared_from_this() { - return std::static_pointer_cast(IExternalLoadable::shared_from_this()); + return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } - std::shared_ptr shared_from_this() const + std::shared_ptr shared_from_this() const { - return std::static_pointer_cast(IExternalLoadable::shared_from_this()); + return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } private: mutable std::mutex name_mutex; - mutable StorageID dict_id; + mutable StorageID dictionary_id; protected: const String full_name; }; -struct IDictionary : IDictionaryBase -{ - IDictionary(const StorageID & dict_id_) : IDictionaryBase(dict_id_) {} - - virtual bool hasHierarchy() const = 0; - - virtual void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const = 0; - - /// TODO: Rewrite - /// Methods for hierarchy. - - virtual void isInVectorVector( - const PaddedPODArray & /*child_ids*/, const PaddedPODArray & /*ancestor_ids*/, PaddedPODArray & /*out*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); - } - - virtual void - isInVectorConstant(const PaddedPODArray & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray & /*out*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); - } - - virtual void - isInConstantVector(const Key /*child_id*/, const PaddedPODArray & /*ancestor_ids*/, PaddedPODArray & /*out*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); - } - - void isInConstantConstant(const Key child_id, const Key ancestor_id, UInt8 & out) const - { - PaddedPODArray out_arr(1); - isInVectorConstant(PaddedPODArray(1, child_id), ancestor_id, out_arr); - out = out_arr[0]; - } -}; - } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 165fa3a000d..4f3773c9300 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -248,7 +248,7 @@ IPAddressDictionary::IPAddressDictionary( DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_) - : IDictionaryBase(dict_id_) + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -857,9 +857,6 @@ static auto keyViewGetter() BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const { - using BlockInputStreamType = DictionaryBlockInputStream; - - const bool is_ipv4 = std::get_if(&ip_column) != nullptr; auto get_keys = [is_ipv4](const Columns & columns, const std::vector & dict_attributes) @@ -880,12 +877,12 @@ BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & colum if (is_ipv4) { auto get_view = keyViewGetter, true>(); - return std::make_shared( + return std::make_shared( shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); } auto get_view = keyViewGetter(); - return std::make_shared( + return std::make_shared( shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); } diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index dcfb26c3c96..cf79caa75fc 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -20,7 +20,7 @@ namespace DB { -class IPAddressDictionary final : public IDictionaryBase +class IPAddressDictionary final : public IDictionary { public: IPAddressDictionary( diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 1d0c75f6bff..dc51bc4b7bd 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -30,7 +30,7 @@ IPolygonDictionary::IPolygonDictionary( const DictionaryLifetime dict_lifetime_, InputType input_type_, PointType point_type_) - : IDictionaryBase(dict_id_) + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) @@ -142,7 +142,6 @@ ColumnPtr IPolygonDictionary::getColumn( callOnDictionaryAttributeType(attribute.underlying_type, type_call); } - query_count.fetch_add(requested_key_points.size(), std::memory_order_relaxed); return result; diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index b82a8b2928f..5974e6461a7 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -24,7 +24,7 @@ namespace bg = boost::geometry; * An implementation should inherit from this base class and preprocess the data upon construction if needed. * It must override the find method of this class which retrieves the polygon containing a single point. */ -class IPolygonDictionary : public IDictionaryBase +class IPolygonDictionary : public IDictionary { public: /** Controls the different types of polygons allowed as input. diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionaryBlockInputStream.h index 6531f5cba9d..499eea7152f 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -24,7 +24,7 @@ public: using Key = UInt64; RangeDictionaryBlockInputStream( - std::shared_ptr dictionary, + std::shared_ptr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray && ids_to_fill, @@ -49,7 +49,7 @@ private: const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const; - std::shared_ptr dictionary; + std::shared_ptr dictionary; NameSet column_names; PaddedPODArray ids; PaddedPODArray start_dates; @@ -59,7 +59,7 @@ private: template RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( - std::shared_ptr dictionary_, + std::shared_ptr dictionary_, size_t max_block_size_, const Names & column_names_, PaddedPODArray && ids_, diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 4196d6ebd72..30395114a8e 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -76,7 +76,7 @@ RangeHashedDictionary::RangeHashedDictionary( DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_) - : IDictionaryBase(dict_id_) + : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) @@ -185,10 +185,10 @@ ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, con auto range_column_storage_type = std::make_shared(); auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type); - PaddedPODArray key_backup_storage; + PaddedPODArray key_backup_storage; PaddedPODArray range_backup_storage; - const PaddedPODArray & ids = getColumnVectorData(this, key_columns[0], key_backup_storage); + const PaddedPODArray & ids = getColumnVectorData(this, key_columns[0], key_backup_storage); const PaddedPODArray & dates = getColumnVectorData(this, range_column_updated, range_backup_storage); const auto & attribute = attributes.front(); @@ -213,7 +213,7 @@ ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Columns & key_columns, con template ColumnUInt8::Ptr RangeHashedDictionary::hasKeysImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & ids, const PaddedPODArray & dates) const { auto result = ColumnUInt8::create(ids.size()); @@ -388,10 +388,10 @@ void RangeHashedDictionary::getItemsImpl( ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const { - PaddedPODArray key_backup_storage; + PaddedPODArray key_backup_storage; PaddedPODArray range_backup_storage; - const PaddedPODArray & ids = getColumnVectorData(this, key_columns[0], key_backup_storage); + const PaddedPODArray & ids = getColumnVectorData(this, key_columns[0], key_backup_storage); const PaddedPODArray & dates = getColumnVectorData(this, key_columns[1], range_backup_storage); const auto & attr = *std::get>(attribute.maps); @@ -436,7 +436,7 @@ void RangeHashedDictionary::getItemsImpl( template -void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const Range & range, const Field & value) +void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const UInt64 id, const Range & range, const Field & value) { using ValueType = std::conditional_t, StringRef, T>; auto & map = *std::get>(attribute.maps); @@ -480,7 +480,7 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K map.insert({id, Values{std::move(value_to_insert)}}); } -void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key id, const Range & range, const Field & value) +void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const UInt64 id, const Range & range, const Field & value) { auto type_call = [&](const auto &dictionary_attribute_type) { @@ -515,7 +515,7 @@ RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name, template void RangeHashedDictionary::getIdsAndDates( - PaddedPODArray & ids, + PaddedPODArray & ids, PaddedPODArray & start_dates, PaddedPODArray & end_dates) const { @@ -536,7 +536,7 @@ void RangeHashedDictionary::getIdsAndDates( template void RangeHashedDictionary::getIdsAndDates( const Attribute & attribute, - PaddedPODArray & ids, + PaddedPODArray & ids, PaddedPODArray & start_dates, PaddedPODArray & end_dates) const { @@ -567,7 +567,7 @@ void RangeHashedDictionary::getIdsAndDates( template BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const { - PaddedPODArray ids; + PaddedPODArray ids; PaddedPODArray start_dates; PaddedPODArray end_dates; getIdsAndDates(ids, start_dates, end_dates); diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index f2b24e52dfc..ca2a925df5e 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -16,7 +16,7 @@ namespace DB { -class RangeHashedDictionary final : public IDictionaryBase +class RangeHashedDictionary final : public IDictionary { public: RangeHashedDictionary( @@ -160,25 +160,25 @@ private: template ColumnUInt8::Ptr hasKeysImpl( const Attribute & attribute, - const PaddedPODArray & ids, + const PaddedPODArray & ids, const PaddedPODArray & dates) const; template - static void setAttributeValueImpl(Attribute & attribute, const Key id, const Range & range, const Field & value); + static void setAttributeValueImpl(Attribute & attribute, const UInt64 id, const Range & range, const Field & value); - static void setAttributeValue(Attribute & attribute, const Key id, const Range & range, const Field & value); + static void setAttributeValue(Attribute & attribute, const UInt64 id, const Range & range, const Field & value); const Attribute & getAttribute(const std::string & attribute_name) const; const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const; template - void getIdsAndDates(PaddedPODArray & ids, PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; + void getIdsAndDates(PaddedPODArray & ids, PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; template void getIdsAndDates( const Attribute & attribute, - PaddedPODArray & ids, + PaddedPODArray & ids, PaddedPODArray & start_dates, PaddedPODArray & end_dates) const; diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index a7b3c87267d..8d24a6ea979 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -57,7 +57,6 @@ void registerDictionaries() { auto & factory = DictionaryFactory::instance(); registerDictionaryRangeHashed(factory); - registerDictionaryComplexKeyHashed(factory); registerDictionaryTrie(factory); registerDictionaryFlat(factory); registerDictionaryHashed(factory); diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 4df58211118..dc58d3f0a14 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -26,7 +26,7 @@ SRCS( CassandraDictionarySource.cpp CassandraHelpers.cpp ClickHouseDictionarySource.cpp - ComplexKeyHashedDictionary.cpp + DictionaryBlockInputStream.cpp DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp DictionarySourceFactory.cpp @@ -48,6 +48,7 @@ SRCS( FlatDictionary.cpp HTTPDictionarySource.cpp HashedDictionary.cpp + HierarchyDictionariesUtils.cpp IPAddressDictionary.cpp LibraryDictionarySource.cpp LibraryDictionarySourceExternal.cpp diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 2c322698327..9d190644b3c 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -28,16 +28,6 @@ #include #include - -#include -#include -#include -#include -#include -#include -#include -#include - #include #include @@ -49,7 +39,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int UNSUPPORTED_METHOD; - extern const int UNKNOWN_TYPE; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; @@ -77,7 +66,7 @@ class FunctionDictHelper public: explicit FunctionDictHelper(const Context & context_) : context(context_) {} - std::shared_ptr getDictionary(const String & dictionary_name) + std::shared_ptr getDictionary(const String & dictionary_name) { auto dict = context.getExternalDictionariesLoader().getDictionary(dictionary_name, context); @@ -90,7 +79,7 @@ public: return dict; } - std::shared_ptr getDictionary(const ColumnWithTypeAndName & column) + std::shared_ptr getDictionary(const ColumnWithTypeAndName & column) { const auto * dict_name_col = checkAndGetColumnConst(column.column.get()); return getDictionary(dict_name_col->getValue()); @@ -744,109 +733,15 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - auto dict = helper.getDictionary(arguments[0]); - ColumnPtr res; + auto dictionary = helper.getDictionary(arguments[0]); - /// TODO: Rewrite this - if (!((res = executeDispatch(arguments, result_type, dict)) - || (res = executeDispatch>(arguments, result_type, dict)) - || (res = executeDispatch(arguments, result_type, dict)) - || (res = executeDispatch>(arguments, result_type, dict)))) - throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; + if (!dictionary->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Dictionary ({}) does not support hierarchy", + dictionary->getFullName()); - return res; - } - - template - ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const std::shared_ptr & dict_ptr) const - { - const auto * dict = typeid_cast(dict_ptr.get()); - if (!dict) - return nullptr; - - if (!dict->hasHierarchy()) - throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD}; - - const auto get_hierarchies = [&] (const PaddedPODArray & in, PaddedPODArray & out, PaddedPODArray & offsets) - { - const auto size = in.size(); - - /// copy of `in` array - auto in_array = std::make_unique>(std::begin(in), std::end(in)); - /// used for storing and handling result of ::toParent call - auto out_array = std::make_unique>(size); - /// resulting hierarchies - std::vector> hierarchies(size); /// TODO Bad code, poor performance. - - /// total number of non-zero elements, used for allocating all the required memory upfront - size_t total_count = 0; - - while (true) - { - auto all_zeroes = true; - - /// erase zeroed identifiers, store non-zeroed ones - for (const auto i : ext::range(0, size)) - { - const auto id = (*in_array)[i]; - if (0 == id) - continue; - - - auto & hierarchy = hierarchies[i]; - - /// Checking for loop - if (std::find(std::begin(hierarchy), std::end(hierarchy), id) != std::end(hierarchy)) - continue; - - all_zeroes = false; - /// place id at it's corresponding place - hierarchy.push_back(id); - - ++total_count; - } - - if (all_zeroes) - break; - - /// translate all non-zero identifiers at once - dict->toParent(*in_array, *out_array); - - /// we're going to use the `in_array` from this iteration as `out_array` on the next one - std::swap(in_array, out_array); - } - - out.reserve(total_count); - offsets.resize(size); - - for (const auto i : ext::range(0, size)) - { - const auto & ids = hierarchies[i]; - out.insert_assume_reserved(std::begin(ids), std::end(ids)); - offsets[i] = out.size(); - } - }; - - const auto * id_col_untyped = arguments[1].column.get(); - if (const auto * id_col = checkAndGetColumn(id_col_untyped)) - { - const auto & in = id_col->getData(); - auto backend = ColumnUInt64::create(); - auto offsets = ColumnArray::ColumnOffsets::create(); - get_hierarchies(in, backend->getData(), offsets->getData()); - return ColumnArray::create(std::move(backend), std::move(offsets)); - } - else if (const auto * id_col_const = checkAndGetColumnConst>(id_col_untyped)) - { - const PaddedPODArray in(1, id_col_const->getValue()); - auto backend = ColumnUInt64::create(); - auto offsets = ColumnArray::ColumnOffsets::create(); - get_hierarchies(in, backend->getData(), offsets->getData()); - auto array = ColumnArray::create(std::move(backend), std::move(offsets)); - return result_type->createColumnConst(id_col_const->size(), (*array)[0].get()); - } - else - throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN}; + ColumnPtr result = dictionary->getHierarchy(arguments[1].column, std::make_shared()); + return result; } mutable FunctionDictHelper helper; @@ -900,105 +795,14 @@ private: auto dict = helper.getDictionary(arguments[0]); - ColumnPtr res; - if (!((res = executeDispatch(arguments, dict)) - || (res = executeDispatch>(arguments, dict)) - || (res = executeDispatch(arguments, dict)) - || (res = executeDispatch>(arguments, dict)))) - throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE}; + if (!dict->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); + + ColumnPtr res = dict->isInHierarchy(arguments[1].column, arguments[2].column, std::make_shared()); return res; } - template - ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const std::shared_ptr & dict_ptr) const - { - const auto * dict = typeid_cast(dict_ptr.get()); - if (!dict) - return nullptr; - - if (!dict->hasHierarchy()) - throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD}; - - const auto * child_id_col_untyped = arguments[1].column.get(); - const auto * ancestor_id_col_untyped = arguments[2].column.get(); - - if (const auto * child_id_col = checkAndGetColumn(child_id_col_untyped)) - return execute(dict, child_id_col, ancestor_id_col_untyped); - else if (const auto * child_id_col_const = checkAndGetColumnConst>(child_id_col_untyped)) - return execute(dict, child_id_col_const, ancestor_id_col_untyped); - else - throw Exception{"Illegal column " + child_id_col_untyped->getName() - + " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - - template - ColumnPtr execute(const DictionaryType * dict, - const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) const - { - if (const auto * ancestor_id_col = checkAndGetColumn(ancestor_id_col_untyped)) - { - auto out = ColumnUInt8::create(); - - const auto & child_ids = child_id_col->getData(); - const auto & ancestor_ids = ancestor_id_col->getData(); - auto & data = out->getData(); - const auto size = child_id_col->size(); - data.resize(size); - - dict->isInVectorVector(child_ids, ancestor_ids, data); - return out; - } - else if (const auto * ancestor_id_col_const = checkAndGetColumnConst>(ancestor_id_col_untyped)) - { - auto out = ColumnUInt8::create(); - - const auto & child_ids = child_id_col->getData(); - const auto ancestor_id = ancestor_id_col_const->getValue(); - auto & data = out->getData(); - const auto size = child_id_col->size(); - data.resize(size); - - dict->isInVectorConstant(child_ids, ancestor_id, data); - return out; - } - else - { - throw Exception{"Illegal column " + ancestor_id_col_untyped->getName() - + " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - } - - template - ColumnPtr execute(const DictionaryType * dict, const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped) const - { - if (const auto * ancestor_id_col = checkAndGetColumn(ancestor_id_col_untyped)) - { - auto out = ColumnUInt8::create(); - - const auto child_id = child_id_col->getValue(); - const auto & ancestor_ids = ancestor_id_col->getData(); - auto & data = out->getData(); - const auto size = child_id_col->size(); - data.resize(size); - - dict->isInConstantVector(child_id, ancestor_ids, data); - return out; - } - else if (const auto * ancestor_id_col_const = checkAndGetColumnConst>(ancestor_id_col_untyped)) - { - const auto child_id = child_id_col->getValue(); - const auto ancestor_id = ancestor_id_col_const->getValue(); - UInt8 res = 0; - - dict->isInConstantConstant(child_id, ancestor_id, res); - return DataTypeUInt8().createColumnConst(child_id_col->size(), res); - } - else - throw Exception{"Illegal column " + ancestor_id_col_untyped->getName() - + " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - mutable FunctionDictHelper helper; }; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 1632b7cbf78..8df29459b72 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -46,13 +46,13 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, const Context & context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, context.getCurrentDatabase()); - return std::static_pointer_cast(load(resolved_dictionary_name)); + return std::static_pointer_cast(load(resolved_dictionary_name)); } ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, const Context & context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, context.getCurrentDatabase()); - return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); + return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); } diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 0f64715b243..ce5b2512741 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -15,7 +15,7 @@ class IExternalLoaderConfigRepository; class ExternalDictionariesLoader : public ExternalLoader { public: - using DictPtr = std::shared_ptr; + using DictPtr = std::shared_ptr; /// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds. explicit ExternalDictionariesLoader(Context & global_context_); diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index cccd23ffbd1..378905b7dc0 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -58,7 +58,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con const auto & external_dictionaries = context.getExternalDictionariesLoader(); for (const auto & load_result : external_dictionaries.getLoadResults()) { - const auto dict_ptr = std::dynamic_pointer_cast(load_result.object); + const auto dict_ptr = std::dynamic_pointer_cast(load_result.object); DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); StorageID dict_id = StorageID::createEmpty(); diff --git a/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.reference b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.reference new file mode 100644 index 00000000000..2cc0a8668a2 --- /dev/null +++ b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.reference @@ -0,0 +1,132 @@ +Dictionary hashed_dictionary_simple_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 value_second_1 +2 value_2 value_second_2 +Dictionary sparse_hashed_dictionary_simple_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 value_second_1 +2 value_2 value_second_2 +Dictionary hashed_dictionary_simple_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 \N +2 value_2 value_second_2 +Dictionary sparse_hashed_dictionary_simple_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 \N +2 value_2 value_second_2 +Dictionary hashed_dictionary_simple_key_hierarchy +dictGet +0 +0 +1 +1 +2 +dictGetHierarchy +[1] +[4,2,1] +Dictionary sparse_hashed_dictionary_simple_key_hierarchy +dictGet +0 +0 +1 +1 +2 +dictGetHierarchy +[1] +[4,2,1] diff --git a/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql new file mode 100644 index 00000000000..7502c6a93bb --- /dev/null +++ b/tests/queries/0_stateless/01765_hashed_dictionary_simple_key.sql @@ -0,0 +1,207 @@ +DROP DATABASE IF EXISTS 01765_db; +CREATE DATABASE 01765_db; + +CREATE TABLE 01765_db.simple_key_simple_attributes_source_table +( + id UInt64, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1'); +INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(HASHED()); + +SELECT 'Dictionary hashed_dictionary_simple_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01765_db.hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01765_db.hashed_dictionary_simple_key_simple_attributes ORDER BY id; + +DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes; + +CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(SPARSE_HASHED()); + +SELECT 'Dictionary sparse_hashed_dictionary_simple_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes ORDER BY id; + +DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes; + +DROP TABLE 01765_db.simple_key_simple_attributes_source_table; + +CREATE TABLE 01765_db.simple_key_complex_attributes_source_table +( + id UInt64, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL); +INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(HASHED()); + +SELECT 'Dictionary hashed_dictionary_simple_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01765_db.hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01765_db.hashed_dictionary_simple_key_complex_attributes ORDER BY id; + +DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes; + +CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(HASHED()); + +SELECT 'Dictionary sparse_hashed_dictionary_simple_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes ORDER BY id; + +DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes; + +DROP TABLE 01765_db.simple_key_complex_attributes_source_table; + +CREATE TABLE 01765_db.simple_key_hierarchy_table +( + id UInt64, + parent_id UInt64 +) ENGINE = TinyLog(); + +INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (1, 0); +INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (2, 1); +INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (3, 1); +INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (4, 2); + +CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(HASHED()); + +SELECT 'Dictionary hashed_dictionary_simple_key_hierarchy'; +SELECT 'dictGet'; +SELECT dictGet('01765_db.hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5; +SELECT 'dictGetHierarchy'; +SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(1)); +SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(4)); + +DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy; + +CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(HASHED()); + +SELECT 'Dictionary sparse_hashed_dictionary_simple_key_hierarchy'; +SELECT 'dictGet'; +SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5; +SELECT 'dictGetHierarchy'; +SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(1)); +SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(4)); + +DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy; + +DROP TABLE 01765_db.simple_key_hierarchy_table; + +DROP DATABASE 01765_db; diff --git a/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.reference b/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.reference new file mode 100644 index 00000000000..12c210581c2 --- /dev/null +++ b/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.reference @@ -0,0 +1,56 @@ +Dictionary hashed_dictionary_complex_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 value_second_1 +2 id_key_2 value_2 value_second_2 +Dictionary hashed_dictionary_complex_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 \N +2 id_key_2 value_2 value_second_2 diff --git a/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.sql b/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.sql new file mode 100644 index 00000000000..de7ab5b5a1a --- /dev/null +++ b/tests/queries/0_stateless/01766_hashed_dictionary_complex_key.sql @@ -0,0 +1,98 @@ +DROP DATABASE IF EXISTS 01766_db; +CREATE DATABASE 01766_db; + +CREATE TABLE 01766_db.complex_key_simple_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1'); +INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes +( + id UInt64, + id_key String, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01766_db')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_HASHED()); + +SELECT 'Dictionary hashed_dictionary_complex_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01766_db.hashed_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01766_db.hashed_dictionary_complex_key_simple_attributes ORDER BY (id, id_key); + +DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes; + +DROP TABLE 01766_db.complex_key_simple_attributes_source_table; + +CREATE TABLE 01766_db.complex_key_complex_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL); +INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes +( + id UInt64, + id_key String, + + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01766_db')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_HASHED()); + +SELECT 'Dictionary hashed_dictionary_complex_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('01766_db.hashed_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM 01766_db.hashed_dictionary_complex_key_complex_attributes ORDER BY (id, id_key); + +DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes; +DROP TABLE 01766_db.complex_key_complex_attributes_source_table; + +DROP DATABASE 01766_db; From bbd180caf3fd20d4d87ec85d82ed103519c55b12 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Mar 2021 22:55:06 +0300 Subject: [PATCH 074/266] Added functions dictGetChildren, dictGetDescendants --- src/Dictionaries/FlatDictionary.cpp | 43 +++++++ src/Dictionaries/FlatDictionary.h | 5 + src/Dictionaries/HierarchyDictionariesUtils.h | 61 ++++++++- src/Dictionaries/IDictionary.h | 14 ++- src/Functions/FunctionsExternalDictionaries.h | 119 ++++++++++++++++++ 5 files changed, 235 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index c67e9686e10..380da1cf2e0 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -221,6 +221,49 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( return result; } +ColumnPtr FlatDictionary::getDescendands( + ColumnPtr key_column, + const DataTypePtr &, + size_t level) const +{ + PaddedPODArray keys_backup; + const auto & keys = getColumnVectorData(this, key_column, keys_backup); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const UInt64 null_value = std::get(hierarchical_attribute.null_values); + const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); + + HashMap parent_to_child; + + for (size_t i = 0; i < parent_keys.size(); ++i) + { + auto parent_key = parent_keys[i]; + parent_to_child[parent_key] = static_cast(i); + } + + auto is_key_valid_func = [&](auto & key) + { + return parent_to_child.find(key) != nullptr; + }; + + auto get_child_key_func = [&](auto & key) + { + std::optional result; + + auto it = parent_to_child.find(key); + + if (it) + result = it->getMapped(); + + return result; + }; + + auto result = getDescendandsArray(keys, null_value, level, is_key_valid_func, get_child_key_func); + return result; +} + void FlatDictionary::createAttributes() { const auto size = dict_struct.attributes.size(); diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index a47ac8c34d8..f4e9ef8a559 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -82,6 +82,11 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; + ColumnPtr getDescendands( + ColumnPtr key_column, + const DataTypePtr & key_type, + size_t level) const override; + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index ba2a74db298..7dd0fd311bc 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -32,17 +32,17 @@ struct IsKeyValidFuncInterface }; template -struct GetParentKeyFuncInterface +struct GetKeyFuncInterface { std::optional operator()(T key [[maybe_unused]]) { return {}; } }; -template +template ElementsAndOffsets getKeysHierarchy( const PaddedPODArray & hierarchy_keys, const KeyType & hierarchy_null_value, IsKeyValidFunc && is_key_valid_func, - GetParentKeyFunc && get_parent_func) + GetKeyFunc && get_key_func) { size_t hierarchy_keys_size = hierarchy_keys.size(); @@ -114,7 +114,7 @@ ElementsAndOffsets getKeysHierarchy( elements.emplace_back(hierarchy_key); ++current_hierarchy_depth; - std::optional parent_key = std::forward(get_parent_func)(hierarchy_key); + std::optional parent_key = std::forward(get_key_func)(hierarchy_key); if (!parent_key.has_value()) break; @@ -162,7 +162,11 @@ PaddedPODArray isInKeysHierarchy( PaddedPODArray result; result.resize_fill(hierarchy_keys.size()); - ElementsAndOffsets hierarchy = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); + ElementsAndOffsets hierarchy = getKeysHierarchy( + hierarchy_keys, + hierarchy_null_value, + std::forward(is_key_valid_func), + std::forward(get_parent_func)); auto & offsets = hierarchy.offsets; auto & elements = hierarchy.elements; @@ -186,6 +190,53 @@ PaddedPODArray isInKeysHierarchy( return result; } +template +ColumnPtr getDescendandsArray( + const PaddedPODArray & hierarchy_keys, + const KeyType & hierarchy_null_value, + size_t level, + IsKeyValidFunc && is_key_valid_func, + GetDescendantKeyFunc && get_descendant_func) +{ + auto elements_and_offsets = getKeysHierarchy( + hierarchy_keys, + hierarchy_null_value, + std::forward(is_key_valid_func), + std::forward(get_descendant_func)); + + auto & elements = elements_and_offsets.elements; + auto & offsets = elements_and_offsets.offsets; + + PaddedPODArray descendants; + descendants.reserve(elements.size()); + + PaddedPODArray descendants_offsets; + descendants_offsets.reserve(elements.size()); + + for (size_t i = 0; i < offsets.size(); ++i) + { + size_t offset_start_index = i > 0 ? offsets[i - 1] : 0; + size_t offset_end_index = offsets[i]; + size_t size = offset_end_index - offset_start_index; + + if (level == 0) + descendants.insert(elements.begin() + offset_start_index + 1, elements.begin() + offset_end_index); + else if (level < size) + descendants.emplace_back(elements[offset_start_index + level]); + + descendants_offsets.emplace_back(descendants.size()); + } + + auto elements_column = ColumnVector::create(); + elements_column->getData() = std::move(elements_and_offsets.elements); + + auto offsets_column = ColumnVector::create(); + offsets_column->getData() = std::move(offsets); + + auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); + return column_array; +} + ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type); ColumnUInt8::Ptr isInHierarchyDefaultImplementation( diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 8b10cd7a819..8bc4e45ffa1 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -165,7 +165,7 @@ struct IDictionary : public IExternalLoadable const DataTypePtr & key_type [[maybe_unused]]) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Hierarchy is not supported for {} dictionary.", + "Method getHierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } @@ -175,7 +175,17 @@ struct IDictionary : public IExternalLoadable const DataTypePtr & key_type [[maybe_unused]]) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Hierarchy is not supported for {} dictionary.", + "Method isInHierarchy is not supported for {} dictionary.", + getDictionaryID().getNameForLogs()); + } + + virtual ColumnPtr getDescendands( + ColumnPtr key_column [[maybe_unused]], + const DataTypePtr & key_type [[maybe_unused]], + size_t level [[maybe_unused]]) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Method getDescendands is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 9d190644b3c..d1f8ff5a1f8 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -806,4 +806,123 @@ private: mutable FunctionDictHelper helper; }; +class FunctionDictGetChildren final : public IFunction +{ +public: + static constexpr auto name = "dictGetChildren"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + explicit FunctionDictGetChildren(const Context & context_) + : helper(context_) {} + + String getName() const override { return name; } + +private: + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const final { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!WhichDataType(arguments[1]).isUInt64()) + throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(std::make_shared()); + } + + bool isDeterministic() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + if (input_rows_count == 0) + return result_type->createColumn(); + + auto dict = helper.getDictionary(arguments[0]); + + if (!dict->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); + + ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared(), 0); + + return res; + } + + mutable FunctionDictHelper helper; +}; + +class FunctionDictGetDescendands final : public IFunction +{ +public: + static constexpr auto name = "dictGetDescendands"; + + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + explicit FunctionDictGetDescendands(const Context & context_) + : helper(context_) {} + + String getName() const override { return name; } + +private: + size_t getNumberOfArguments() const override { return 3; } + + bool useDefaultImplementationForConstants() const final { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 2}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!WhichDataType(arguments[1]).isUInt64()) + throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!WhichDataType(arguments[2]).isUInt64()) + throw Exception{"Illegal type " + arguments[1]->getName() + " of third argument of function " + getName() + + ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + if (input_rows_count == 0) + return result_type->createColumn(); + + const auto * level_const_column = checkAndGetColumnConst>(arguments[2].column.get()); + + if (!level_const_column) + throw Exception{"Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName() + + ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + auto dict = helper.getDictionary(arguments[0]); + + if (!dict->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); + + size_t level = static_cast(level_const_column->getValue()); + ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared(), level); + + return res; + } + + mutable FunctionDictHelper helper; +}; + } From 566541528d948d4e3fbf7fe8b44ea2cf13cd3933 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 10:31:12 +0300 Subject: [PATCH 075/266] Update function naming --- src/Dictionaries/FlatDictionary.cpp | 12 ++++++++-- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HierarchyDictionariesUtils.h | 16 ++++++++++--- src/Dictionaries/IDictionary.h | 4 ++-- .../FunctionsExternalDictionaries.cpp | 2 ++ src/Functions/FunctionsExternalDictionaries.h | 23 ++++++++----------- 6 files changed, 37 insertions(+), 22 deletions(-) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 380da1cf2e0..495f3678f86 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -221,7 +221,7 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( return result; } -ColumnPtr FlatDictionary::getDescendands( +ColumnPtr FlatDictionary::getDescendants( ColumnPtr key_column, const DataTypePtr &, size_t level) const @@ -243,6 +243,14 @@ ColumnPtr FlatDictionary::getDescendands( parent_to_child[parent_key] = static_cast(i); } + + std::cerr << "FlatDictionary::getDescendants " << parent_to_child.size() << std::endl; + for (auto & node : parent_to_child) + { + std::cerr << node.getKey() << " " << node.getMapped() << std::endl; + } + std::cerr << std::endl; + auto is_key_valid_func = [&](auto & key) { return parent_to_child.find(key) != nullptr; @@ -260,7 +268,7 @@ ColumnPtr FlatDictionary::getDescendands( return result; }; - auto result = getDescendandsArray(keys, null_value, level, is_key_valid_func, get_child_key_func); + auto result = getDescendantsArray(keys, null_value, level, is_key_valid_func, get_child_key_func); return result; } diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index f4e9ef8a559..09721bf1a99 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -82,7 +82,7 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; - ColumnPtr getDescendands( + ColumnPtr getDescendants( ColumnPtr key_column, const DataTypePtr & key_type, size_t level) const override; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 7dd0fd311bc..e8f6631f6ab 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -191,7 +191,7 @@ PaddedPODArray isInKeysHierarchy( } template -ColumnPtr getDescendandsArray( +ColumnPtr getDescendantsArray( const PaddedPODArray & hierarchy_keys, const KeyType & hierarchy_null_value, size_t level, @@ -207,6 +207,16 @@ ColumnPtr getDescendandsArray( auto & elements = elements_and_offsets.elements; auto & offsets = elements_and_offsets.offsets; + std::cerr << "getDescendantsArray" << std::endl; + std::cerr << "Elements " << elements.size() << std::endl; + for (auto element : elements) + std::cerr << element << " "; + std::cerr << std::endl; + std::cerr << "Offsets " << offsets.size() << std::endl; + for (auto offset : offsets) + std::cerr << offset << " "; + std::cerr << std::endl; + PaddedPODArray descendants; descendants.reserve(elements.size()); @@ -228,10 +238,10 @@ ColumnPtr getDescendandsArray( } auto elements_column = ColumnVector::create(); - elements_column->getData() = std::move(elements_and_offsets.elements); + elements_column->getData() = std::move(descendants); auto offsets_column = ColumnVector::create(); - offsets_column->getData() = std::move(offsets); + offsets_column->getData() = std::move(descendants_offsets); auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); return column_array; diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 8bc4e45ffa1..a7445312409 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -179,13 +179,13 @@ struct IDictionary : public IExternalLoadable getDictionaryID().getNameForLogs()); } - virtual ColumnPtr getDescendands( + virtual ColumnPtr getDescendants( ColumnPtr key_column [[maybe_unused]], const DataTypePtr & key_type [[maybe_unused]], size_t level [[maybe_unused]]) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Method getDescendands is not supported for {} dictionary.", + "Method getDescendants is not supported for {} dictionary.", getDictionaryID().getNameForLogs()); } diff --git a/src/Functions/FunctionsExternalDictionaries.cpp b/src/Functions/FunctionsExternalDictionaries.cpp index f037a3bd808..79236fb2cf4 100644 --- a/src/Functions/FunctionsExternalDictionaries.cpp +++ b/src/Functions/FunctionsExternalDictionaries.cpp @@ -24,6 +24,8 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index d1f8ff5a1f8..333a4515fe4 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -852,7 +852,7 @@ private: if (!dict->hasHierarchy()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); - ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared(), 0); + ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), 0); return res; } @@ -860,17 +860,17 @@ private: mutable FunctionDictHelper helper; }; -class FunctionDictGetDescendands final : public IFunction +class FunctionDictGetDescendants final : public IFunction { public: - static constexpr auto name = "dictGetDescendands"; + static constexpr auto name = "dictGetDescendants"; static FunctionPtr create(const Context & context) { - return std::make_shared(context); + return std::make_shared(context); } - explicit FunctionDictGetDescendands(const Context & context_) + explicit FunctionDictGetDescendants(const Context & context_) : helper(context_) {} String getName() const override { return name; } @@ -891,9 +891,9 @@ private: throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (!WhichDataType(arguments[2]).isUInt64()) + if (!isUnsignedInteger(arguments[2])) throw Exception{"Illegal type " + arguments[1]->getName() + " of third argument of function " + getName() - + ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + ", must be const unsigned integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; return std::make_shared(); } @@ -905,19 +905,14 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - const auto * level_const_column = checkAndGetColumnConst>(arguments[2].column.get()); - - if (!level_const_column) - throw Exception{"Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName() - + ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + size_t level = static_cast(arguments[2].column->get64(0)); auto dict = helper.getDictionary(arguments[0]); if (!dict->hasHierarchy()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); - size_t level = static_cast(level_const_column->getValue()); - ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared(), level); + ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), level); return res; } From 720e2e0501c9336d505751cbaf9dd8bf1a6f040e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 16:23:19 +0300 Subject: [PATCH 076/266] Updated dictGetDescendants, dictGetChildren implementation --- src/Dictionaries/FlatDictionary.cpp | 36 +- src/Dictionaries/HashedDictionary.cpp | 28 ++ src/Dictionaries/HashedDictionary.h | 5 + .../HierarchyDictionariesUtils.cpp | 2 - src/Dictionaries/HierarchyDictionariesUtils.h | 432 ++++++++++++------ src/Functions/FunctionsExternalDictionaries.h | 4 +- 6 files changed, 327 insertions(+), 180 deletions(-) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 495f3678f86..ce9b2792688 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -218,6 +218,8 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( auto result = ColumnUInt8::create(); result->getData() = std::move(is_in_hierarchy_result); + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + return result; } @@ -231,44 +233,22 @@ ColumnPtr FlatDictionary::getDescendants( size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; - - const UInt64 null_value = std::get(hierarchical_attribute.null_values); const ContainerType & parent_keys = std::get>(hierarchical_attribute.arrays); - HashMap parent_to_child; + HashMap> parent_to_child; for (size_t i = 0; i < parent_keys.size(); ++i) { auto parent_key = parent_keys[i]; - parent_to_child[parent_key] = static_cast(i); + + if (loaded_ids[i]) + parent_to_child[parent_key].emplace_back(static_cast(i)); } + auto result = getDescendantsArray(keys, parent_to_child, level); - std::cerr << "FlatDictionary::getDescendants " << parent_to_child.size() << std::endl; - for (auto & node : parent_to_child) - { - std::cerr << node.getKey() << " " << node.getMapped() << std::endl; - } - std::cerr << std::endl; + query_count.fetch_add(keys.size(), std::memory_order_relaxed); - auto is_key_valid_func = [&](auto & key) - { - return parent_to_child.find(key) != nullptr; - }; - - auto get_child_key_func = [&](auto & key) - { - std::optional result; - - auto it = parent_to_child.find(key); - - if (it) - result = it->getMapped(); - - return result; - }; - - auto result = getDescendantsArray(keys, null_value, level, is_key_valid_func, get_child_key_func); return result; } diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index b5cb6b43396..5729df8dd93 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -287,6 +287,34 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( return nullptr; } +template +ColumnPtr HashedDictionary::getDescendants( + ColumnPtr key_column, + const DataTypePtr &, + size_t level) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::simple) + { + PaddedPODArray keys_backup; + const auto & keys = getColumnVectorData(this, key_column, keys_backup); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + const CollectionType & parent_keys = std::get>(hierarchical_attribute.container); + + HashMap> parent_to_child; + + for (const auto & [key, value] : parent_keys) + parent_to_child[value].emplace_back(key); + + auto result = getDescendantsArray(keys, parent_to_child, level); + return result; + } + else + return nullptr; +} + template void HashedDictionary::createAttributes() { diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index a3290c800d1..d9568727fb9 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -102,6 +102,11 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; + ColumnPtr getDescendants( + ColumnPtr key_column, + const DataTypePtr & key_type, + size_t level) const override; + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index c7e22419fc0..8839e383843 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -1,7 +1,5 @@ #include "HierarchyDictionariesUtils.h" -#include - namespace DB { diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index e8f6631f6ab..d5fdb7ef366 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -18,116 +19,282 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -template -struct ElementsAndOffsets +namespace detail { - PaddedPODArray elements; - PaddedPODArray offsets; -}; - -template -struct IsKeyValidFuncInterface -{ - bool operator()(T key [[maybe_unused]]) { return false; } -}; - -template -struct GetKeyFuncInterface -{ - std::optional operator()(T key [[maybe_unused]]) { return {}; } -}; - -template -ElementsAndOffsets getKeysHierarchy( - const PaddedPODArray & hierarchy_keys, - const KeyType & hierarchy_null_value, - IsKeyValidFunc && is_key_valid_func, - GetKeyFunc && get_key_func) -{ - size_t hierarchy_keys_size = hierarchy_keys.size(); - - PaddedPODArray elements; - elements.reserve(hierarchy_keys_size); - - PaddedPODArray offsets; - offsets.reserve(hierarchy_keys_size); - - struct OffsetInArray + template + struct ElementsAndOffsets { - size_t offset_index; - size_t array_element_offset; + PaddedPODArray elements; + PaddedPODArray offsets; }; - HashMap already_processes_keys_to_offset; - already_processes_keys_to_offset.reserve(hierarchy_keys_size); - - for (size_t i = 0; i < hierarchy_keys_size; ++i) + template + struct IsKeyValidFuncInterface { - auto hierarchy_key = hierarchy_keys[i]; - size_t current_hierarchy_depth = 0; + bool operator()(T key [[maybe_unused]]) { return false; } + }; - bool is_key_valid = std::forward(is_key_valid_func)(hierarchy_key); + template + struct GetParentKeyFuncInterface + { + std::optional operator()(T key [[maybe_unused]]) { return {}; } + }; - if (!is_key_valid) + template + ElementsAndOffsets getKeysHierarchy( + const PaddedPODArray & hierarchy_keys, + const KeyType & hierarchy_null_value, + IsKeyValidFunc && is_key_valid_func, + GetParentKeyFunc && get_key_func) + { + size_t hierarchy_keys_size = hierarchy_keys.size(); + + PaddedPODArray elements; + elements.reserve(hierarchy_keys_size); + + PaddedPODArray offsets; + offsets.reserve(hierarchy_keys_size); + + struct OffsetInArray { - offsets.emplace_back(elements.size()); - continue; - } + size_t offset_index; + size_t array_element_offset; + }; - while (true) + HashMap already_processes_keys_to_offset; + already_processes_keys_to_offset.reserve(hierarchy_keys_size); + + for (size_t i = 0; i < hierarchy_keys_size; ++i) { - const auto * it = already_processes_keys_to_offset.find(hierarchy_key); + auto hierarchy_key = hierarchy_keys[i]; + size_t current_hierarchy_depth = 0; - if (it) + bool is_key_valid = std::forward(is_key_valid_func)(hierarchy_key); + + if (!is_key_valid) { - const auto & index = it->getMapped(); - - size_t offset = index.offset_index; - - bool is_loop = (offset == offsets.size()); - - if (unlikely(is_loop)) - break; - - size_t array_element_offset = index.array_element_offset; - - size_t previous_offset_size = offset > 0 ? offsets[offset - 1] : 0; - size_t start_index = previous_offset_size + array_element_offset; - size_t end_index = offsets[offset]; - - current_hierarchy_depth += end_index - start_index; - - /// TODO: Insert part of pod array into itself - while (start_index < end_index) - { - elements.emplace_back(elements[start_index]); - ++start_index; - } - - break; + offsets.emplace_back(elements.size()); + continue; } - if (hierarchy_key == hierarchy_null_value || current_hierarchy_depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH) - break; + while (true) + { + const auto * it = already_processes_keys_to_offset.find(hierarchy_key); - already_processes_keys_to_offset[hierarchy_key] = {offsets.size(), current_hierarchy_depth}; - elements.emplace_back(hierarchy_key); - ++current_hierarchy_depth; + if (it) + { + const auto & index = it->getMapped(); - std::optional parent_key = std::forward(get_key_func)(hierarchy_key); + size_t offset = index.offset_index; - if (!parent_key.has_value()) - break; + bool is_loop = (offset == offsets.size()); - hierarchy_key = *parent_key; + if (unlikely(is_loop)) + break; + + size_t array_element_offset = index.array_element_offset; + + size_t previous_offset_size = offset > 0 ? offsets[offset - 1] : 0; + size_t start_index = previous_offset_size + array_element_offset; + size_t end_index = offsets[offset]; + + current_hierarchy_depth += end_index - start_index; + + /// TODO: Insert part of pod array into itself + while (start_index < end_index) + { + elements.emplace_back(elements[start_index]); + ++start_index; + } + + break; + } + + if (hierarchy_key == hierarchy_null_value || current_hierarchy_depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH) + break; + + already_processes_keys_to_offset[hierarchy_key] = {offsets.size(), current_hierarchy_depth}; + elements.emplace_back(hierarchy_key); + ++current_hierarchy_depth; + + std::optional parent_key = std::forward(get_key_func)(hierarchy_key); + + if (!parent_key.has_value()) + break; + + hierarchy_key = *parent_key; + } + + offsets.emplace_back(elements.size()); } - offsets.emplace_back(elements.size()); + ElementsAndOffsets result = {std::move(elements), std::move(offsets)}; + + return result; } - ElementsAndOffsets result = {std::move(elements), std::move(offsets)}; + struct GetAllDescendantsStrategy { size_t level = 0; }; + struct GetDescendantsAtSpecificLevelStrategy { size_t level = 0; }; - return result; + template + ElementsAndOffsets getDescendants( + const PaddedPODArray & requested_keys, + const HashMap> & parent_to_child, + Strategy strategy) + { + PaddedPODArray descendants; + descendants.reserve(requested_keys.size()); + + PaddedPODArray descendants_offsets; + descendants_offsets.reserve(requested_keys.size()); + + struct Range + { + size_t start_index; + size_t end_index; + }; + + static constexpr Int64 key_range_requires_update = -1; + HashMap already_processed_keys_to_range [[maybe_unused]]; + + if constexpr (std::is_same_v) + already_processed_keys_to_range.reserve(requested_keys.size()); + + struct KeyAndDepth + { + KeyType key; + Int64 depth; + }; + + HashSet already_processed_keys_during_loop; + already_processed_keys_during_loop.reserve(requested_keys.size()); + + PaddedPODArray next_keys_to_process_stack; + next_keys_to_process_stack.reserve(requested_keys.size()); + + Int64 level = static_cast(strategy.level); + + for (size_t i = 0; i < requested_keys.size(); ++i) + { + const KeyType & requested_key = requested_keys[i]; + + if (parent_to_child.find(requested_key) == nullptr) + { + descendants_offsets.emplace_back(descendants.size()); + continue; + } + + next_keys_to_process_stack.emplace_back(KeyAndDepth{requested_key, 0}); + + while (!next_keys_to_process_stack.empty()) + { + KeyAndDepth key_to_process = next_keys_to_process_stack.back(); + + KeyType key = key_to_process.key; + Int64 depth = key_to_process.depth; + next_keys_to_process_stack.pop_back(); + + if constexpr (std::is_same_v) + { + if (depth == key_range_requires_update) + { + auto * it = already_processed_keys_to_range.find(key); + assert(it); + + auto & range_to_update = it->getMapped(); + range_to_update.end_index = descendants.size(); + continue; + } + } + + if (unlikely(already_processed_keys_during_loop.find(key) != nullptr)) + { + next_keys_to_process_stack.clear(); + break; + } + + if constexpr (std::is_same_v) + { + const auto * already_processed_it = already_processed_keys_to_range.find(key); + + if (already_processed_it) + { + Range range = already_processed_it->getMapped(); + + if (unlikely(range.start_index > range.end_index)) + { + /// Broken range because there was loop + already_processed_keys_to_range.erase(key); + } + else + { + /// TODO: Insert part of pod array + while (range.start_index != range.end_index) + { + descendants.emplace_back(descendants[range.start_index]); + ++range.start_index; + } + + continue; + } + } + } + + const auto * it = parent_to_child.find(key); + + if (!it || depth >= DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH) + continue; + + if constexpr (std::is_same_v) + { + if (depth > level) + continue; + } + + if constexpr (std::is_same_v) + { + size_t range_start_index = descendants.size(); + already_processed_keys_to_range[key].start_index = range_start_index; + next_keys_to_process_stack.emplace_back(KeyAndDepth{key, -1}); + } + + already_processed_keys_during_loop.insert(key); + + ++depth; + + const auto & childs = it->getMapped(); + + for (auto child_key : childs) + { + if (std::is_same_v || depth == level) + descendants.emplace_back(child_key); + + next_keys_to_process_stack.emplace_back(KeyAndDepth{child_key, depth}); + } + } + + already_processed_keys_during_loop.clear(); + + descendants_offsets.emplace_back(descendants.size()); + } + + ElementsAndOffsets result = {std::move(descendants), std::move(descendants_offsets)}; + return result; + } + + template + ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets && elements_and_offsets) + { + auto elements_column = ColumnVector::create(); + elements_column->getData() = std::move(elements_and_offsets.elements); + + auto offsets_column = ColumnVector::create(); + offsets_column->getData() = std::move(elements_and_offsets.offsets); + + auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); + + return column_array; + } } template @@ -137,16 +304,8 @@ ColumnPtr getKeysHierarchyArray( IsKeyValidFunc && is_key_valid_func, GetParentKeyFunc && get_parent_func) { - auto elements_and_offsets = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); - - auto elements_column = ColumnVector::create(); - elements_column->getData() = std::move(elements_and_offsets.elements); - - auto offsets_column = ColumnVector::create(); - offsets_column->getData() = std::move(elements_and_offsets.offsets); - - auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); - return column_array; + auto elements_and_offsets = detail::getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); + return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets)); } template @@ -162,7 +321,7 @@ PaddedPODArray isInKeysHierarchy( PaddedPODArray result; result.resize_fill(hierarchy_keys.size()); - ElementsAndOffsets hierarchy = getKeysHierarchy( + detail::ElementsAndOffsets hierarchy = detail::getKeysHierarchy( hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), @@ -190,64 +349,41 @@ PaddedPODArray isInKeysHierarchy( return result; } -template +template ColumnPtr getDescendantsArray( - const PaddedPODArray & hierarchy_keys, - const KeyType & hierarchy_null_value, - size_t level, - IsKeyValidFunc && is_key_valid_func, - GetDescendantKeyFunc && get_descendant_func) + const PaddedPODArray & requested_keys, + const HashMap> & parent_to_child, + size_t level) { - auto elements_and_offsets = getKeysHierarchy( - hierarchy_keys, - hierarchy_null_value, - std::forward(is_key_valid_func), - std::forward(get_descendant_func)); - - auto & elements = elements_and_offsets.elements; - auto & offsets = elements_and_offsets.offsets; - - std::cerr << "getDescendantsArray" << std::endl; - std::cerr << "Elements " << elements.size() << std::endl; - for (auto element : elements) - std::cerr << element << " "; - std::cerr << std::endl; - std::cerr << "Offsets " << offsets.size() << std::endl; - for (auto offset : offsets) - std::cerr << offset << " "; - std::cerr << std::endl; - - PaddedPODArray descendants; - descendants.reserve(elements.size()); - - PaddedPODArray descendants_offsets; - descendants_offsets.reserve(elements.size()); - - for (size_t i = 0; i < offsets.size(); ++i) + for (auto & node : parent_to_child) { - size_t offset_start_index = i > 0 ? offsets[i - 1] : 0; - size_t offset_end_index = offsets[i]; - size_t size = offset_end_index - offset_start_index; - - if (level == 0) - descendants.insert(elements.begin() + offset_start_index + 1, elements.begin() + offset_end_index); - else if (level < size) - descendants.emplace_back(elements[offset_start_index + level]); - - descendants_offsets.emplace_back(descendants.size()); + const auto & key = node.getKey(); + const auto & childs = node.getMapped(); + std::cerr << "Key " << key << " childs " << childs.size() << std::endl; + for (auto child : childs) + std::cerr << child << " "; + std::cerr << std::endl; } - auto elements_column = ColumnVector::create(); - elements_column->getData() = std::move(descendants); - auto offsets_column = ColumnVector::create(); - offsets_column->getData() = std::move(descendants_offsets); - - auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column)); - return column_array; + if (level == 0) + { + detail::GetAllDescendantsStrategy strategy { .level = level }; + auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy); + return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets)); + } + else + { + detail::GetDescendantsAtSpecificLevelStrategy strategy { .level = level }; + auto elements_and_offsets = detail::getDescendants(requested_keys, parent_to_child, strategy); + return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets)); + } } -ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type); +ColumnPtr getHierarchyDefaultImplementation( + const IDictionary * dictionary, + ColumnPtr key_column, + const DataTypePtr & key_type); ColumnUInt8::Ptr isInHierarchyDefaultImplementation( const IDictionary * dictionary, diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 333a4515fe4..b57ff369779 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -852,7 +852,7 @@ private: if (!dict->hasHierarchy()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); - ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), 0); + ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), 1); return res; } @@ -895,7 +895,7 @@ private: throw Exception{"Illegal type " + arguments[1]->getName() + " of third argument of function " + getName() + ", must be const unsigned integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - return std::make_shared(); + return std::make_shared(std::make_shared()); } bool isDeterministic() const override { return false; } From 3b10043eb266b7184e7a58b6802d8a8dd30a268f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 16:24:43 +0300 Subject: [PATCH 077/266] Fix style check --- src/Dictionaries/HierarchyDictionariesUtils.h | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index d5fdb7ef366..0589662e0bb 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -14,11 +14,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int UNSUPPORTED_METHOD; -} - namespace detail { template @@ -355,17 +350,6 @@ ColumnPtr getDescendantsArray( const HashMap> & parent_to_child, size_t level) { - for (auto & node : parent_to_child) - { - const auto & key = node.getKey(); - const auto & childs = node.getMapped(); - std::cerr << "Key " << key << " childs " << childs.size() << std::endl; - for (auto child : childs) - std::cerr << child << " "; - std::cerr << std::endl; - } - - if (level == 0) { detail::GetAllDescendantsStrategy strategy { .level = level }; From 21d28a37aa461af711663a843b7536ed886eb23f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 16:32:25 +0300 Subject: [PATCH 078/266] Fixed build --- src/Dictionaries/CacheDictionary.cpp | 9 +++++++-- src/Dictionaries/HashedDictionary.cpp | 12 ++++++------ src/Dictionaries/HashedDictionary.h | 2 +- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 2c9d6ca764d..26f44dd42c5 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -368,7 +368,9 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k } template -ColumnPtr CacheDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const +ColumnPtr CacheDictionary::getHierarchy( + ColumnPtr key_column [[maybe_unused]], + const DataTypePtr & key_type [[maybe_unused]]) const { if (dictionary_key_type == DictionaryKeyType::simple) { @@ -381,7 +383,10 @@ ColumnPtr CacheDictionary::getHierarchy(ColumnPtr key_colum } template -ColumnUInt8::Ptr CacheDictionary::isInHierarchy(ColumnPtr key_column, ColumnPtr in_key_column, const DataTypePtr & key_type) const +ColumnUInt8::Ptr CacheDictionary::isInHierarchy( + ColumnPtr key_column [[maybe_unused]], + ColumnPtr in_key_column [[maybe_unused]], + const DataTypePtr & key_type [[maybe_unused]]) const { if (dictionary_key_type == DictionaryKeyType::simple) { diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 5729df8dd93..8dad43af17c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -191,7 +191,7 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co } template -ColumnPtr HashedDictionary::getHierarchy(ColumnPtr key_column, const DataTypePtr &) const +ColumnPtr HashedDictionary::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const { if constexpr (dictionary_key_type == DictionaryKeyType::simple) { @@ -200,8 +200,8 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; - auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; - auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; const UInt64 null_value = dictionary_attribute.null_value.get(); const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); @@ -237,8 +237,8 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr template ColumnUInt8::Ptr HashedDictionary::isInHierarchy( - ColumnPtr key_column, - ColumnPtr in_key_column, + ColumnPtr key_column [[maybe_unused]], + ColumnPtr in_key_column [[maybe_unused]], const DataTypePtr &) const { if constexpr (dictionary_key_type == DictionaryKeyType::simple) @@ -289,7 +289,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( template ColumnPtr HashedDictionary::getDescendants( - ColumnPtr key_column, + ColumnPtr key_column [[maybe_unused]], const DataTypePtr &, size_t level) const { diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index d9568727fb9..febab94b255 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -95,7 +95,7 @@ public: bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); } - ColumnPtr getHierarchy(ColumnPtr hierarchy_attribute_column, const DataTypePtr & hierarchy_attribute_type) const override; + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; ColumnUInt8::Ptr isInHierarchy( ColumnPtr key_column, From 57d9939aeedf18011c398cf7543a2190c7002834 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 16:43:54 +0300 Subject: [PATCH 079/266] Fixed tests --- .../0_stateless/01251_dict_is_in_infinite_loop.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference index 757d2858524..0a2c97efb42 100644 --- a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.reference @@ -29,10 +29,10 @@ 1 1 1 -255 -255 0 -255 +0 +0 +0 [11,22] [22,11] [11,22] From eb0039ed030c7f98adc7b2a8ce9f752f6d0c81db Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Mar 2021 20:38:30 +0300 Subject: [PATCH 080/266] Fixed tests --- src/Common/HashTable/HashMap.h | 44 +++- src/Dictionaries/DictionaryHelpers.h | 37 ++- src/Dictionaries/HashedDictionary.cpp | 217 ++++++++++++++---- src/Dictionaries/HashedDictionary.h | 24 +- .../HierarchyDictionariesUtils.cpp | 4 + 5 files changed, 260 insertions(+), 66 deletions(-) diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index 99dc5414107..cfa159cfdc6 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -48,7 +48,7 @@ struct HashMapCell value_type value; - HashMapCell() {} + HashMapCell() = default; HashMapCell(const Key & key_, const State &) : value(key_, NoInitTag()) {} HashMapCell(const value_type & value_, const State &) : value(value_) {} @@ -114,8 +114,38 @@ struct HashMapCell static void move(HashMapCell * /* old_location */, HashMapCell * /* new_location */) {} + template + auto & get() & { + if constexpr (I == 0) return value.first; + else if constexpr (I == 1) return value.second; + } + + template + auto const & get() const & { + if constexpr (I == 0) return value.first; + else if constexpr (I == 1) return value.second; + } + + template + auto && get() && { + if constexpr (I == 0) return std::move(value.first); + else if constexpr (I == 1) return std::move(value.second); + } + }; +namespace std { + + template + struct tuple_size> : std::integral_constant { }; + + template + struct tuple_element<0, HashMapCell> { using type = Key; }; + + template + struct tuple_element<1, HashMapCell> { using type = TMapped; }; +} + template struct HashMapCellWithSavedHash : public HashMapCell { @@ -227,6 +257,18 @@ public: } }; +namespace std { + + template + struct tuple_size> : std::integral_constant { }; + + template + struct tuple_element<0, HashMapCellWithSavedHash> { using type = Key; }; + + template + struct tuple_element<1, HashMapCellWithSavedHash> { using type = TMapped; }; +} + template < typename Key, diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index bb0eba40159..21ebe3e4199 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -307,33 +307,27 @@ public: assert(!key_columns.empty()); if constexpr (key_type == DictionaryKeyType::simple) - keys = getColumnVectorData(key_columns.front()); - else - keys = deserializeKeyColumnsInArena(key_columns, existing_arena); - } + { + simple_key_column = key_columns.front()->convertToFullColumnIfConst(); + const auto * vector_col = checkAndGetColumn>(simple_key_column.get()); + + if (!vector_col) + throw Exception{ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"}; + } + else + complex_keys_serialized = deserializeKeyColumnsInArena(key_columns, existing_arena); + } const PaddedPODArray & getKeys() const { - return keys; + if constexpr (key_type == DictionaryKeyType::simple) + return static_cast *>(simple_key_column.get())->getData(); + else + return complex_keys_serialized; } private: - static PaddedPODArray getColumnVectorData(const ColumnPtr column) - { - PaddedPODArray result; - - auto full_column = column->convertToFullColumnIfConst(); - const auto *vector_col = checkAndGetColumn>(full_column.get()); - - if (!vector_col) - throw Exception{ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"}; - - result.assign(vector_col->getData()); - - return result; - } - static PaddedPODArray deserializeKeyColumnsInArena(const Columns & key_columns, Arena & temporary_arena) { size_t keys_size = key_columns.front()->size(); @@ -361,7 +355,8 @@ private: return result; } - PaddedPODArray keys; + PaddedPODArray complex_keys_serialized; + ColumnPtr simple_key_column; }; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 8dad43af17c..7ebc43e1a72 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -12,6 +12,21 @@ #include #include +namespace +{ + +/// NOTE: Trailing return type is explicitly specified for SFINAE. + +/// google::sparse_hash_map +template auto getKeyFromCell(const T & value) -> decltype(value->first) { return value->first; } // NOLINT +template auto getValueFromCell(const T & value) -> decltype(value->second) { return value->second; } // NOLINT + +/// HashMap +template auto getKeyFromCell(const T & value) -> decltype(value->getKey()) { return value->getKey(); } // NOLINT +template auto getValueFromCell(const T & value) -> decltype(value->getMapped()) { return value->getMapped(); } // NOLINT + +} + namespace DB { @@ -47,9 +62,91 @@ ColumnPtr HashedDictionary::getColumn( const std::string & attribute_name, const DataTypePtr & result_type, const Columns & key_columns, - const DataTypes & key_types, + const DataTypes & key_types [[maybe_unused]], const ColumnPtr & default_values_column) const { + // if constexpr (dictionary_key_type == DictionaryKeyType::simple) + // { + // ColumnPtr result; + + // PaddedPODArray backup_storage; + // const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage); + + // auto size = ids.size(); + + // const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + // DefaultValueProvider default_value_provider(dictionary_attribute.null_value, default_values_column); + + // size_t index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + // const auto & attribute = attributes[index]; + + // auto type_call = [&](const auto & dictionary_attribute_type) + // { + // using Type = std::decay_t; + // using AttributeType = typename Type::AttributeType; + // using ValueType = DictionaryValueType; + // using ColumnProvider = DictionaryAttributeColumnProvider; + + // auto & container = std::get>(attribute.container); + + // auto column = ColumnProvider::getColumn(dictionary_attribute, size); + + // if constexpr (std::is_same_v) + // { + // auto * out = column.get(); + + // const auto rows = ext::size(ids); + + // for (const auto i : ext::range(0, rows)) + // { + // const auto it = container.find(ids[i]); + + // if (it != container.end()) + // { + // StringRef item = getValueFromCell(it); + // out->insertData(item.data, item.size); + // } + // else + // { + // Field default_value = default_value_provider.getDefaultValue(i); + // String & default_value_string = default_value.get(); + // out->insertData(default_value_string.data(), default_value_string.size()); + // } + // } + // } + // else + // { + // auto & out = column->getData(); + // const auto rows = ext::size(ids); + + // for (const auto i : ext::range(0, rows)) + // { + // const auto it = container.find(ids[i]); + + // if (it != container.end()) + // { + // auto item = getValueFromCell(it); + // out[i] = item; + // } + // else + // { + // Field default_value = default_value_provider.getDefaultValue(i); + // out[i] = default_value.get>(); + // } + // } + + // } + + // result = std::move(column); + // }; + + // callOnDictionaryAttributeType(attribute.type, type_call); + // query_count.fetch_add(ids.size(), std::memory_order_relaxed); + + // return result; + // } + // else + // return nullptr; if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); @@ -62,24 +159,25 @@ ColumnPtr HashedDictionary::getColumn( const auto & requested_keys = extractor.getKeys(); auto result_column = dictionary_attribute.type->createColumn(); - result_column->reserve(requested_keys.size()); size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; const auto & attribute = attributes[attribute_index]; - Field row_value_to_insert; + size_t requested_keys_size = requested_keys.size(); if (unlikely(attribute.is_complex_type)) { - auto & attribute_container = std::get>(attribute.container); + const auto & attribute_container = std::get>(attribute.container); - for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) + Field row_value_to_insert; + + for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) { auto & requested_key = requested_keys[requested_key_index]; auto it = attribute_container.find(requested_key); if (it != attribute_container.end()) - row_value_to_insert = it->second; + row_value_to_insert = getValueFromCell(it); else row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); @@ -98,46 +196,51 @@ ColumnPtr HashedDictionary::getColumn( ColumnString, std::conditional_t, ColumnDecimal, ColumnVector>>; - auto & attribute_container = std::get>(attribute.container); + const auto & attribute_container = std::get>(attribute.container); ColumnType & result_column_typed = static_cast(*result_column); if constexpr (std::is_same_v) { - for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) + result_column_typed.reserve(requested_keys_size); + + for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) { auto & requested_key = requested_keys[requested_key_index]; auto it = attribute_container.find(requested_key); if (it != attribute_container.end()) { - auto item = it->second; + StringRef item = getValueFromCell(it); result_column->insertData(item.data, item.size); } else { - row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); - result_column->insert(row_value_to_insert); + Field default_value = default_value_provider.getDefaultValue(requested_key_index); + String & default_value_string = default_value.get(); + result_column->insertData(default_value_string.data(), default_value_string.size()); } } } else { auto & result_data = result_column_typed.getData(); + result_data.resize_fill(requested_keys_size); - for (size_t requested_key_index = 0; requested_key_index < requested_keys.size(); ++requested_key_index) + for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) { auto & requested_key = requested_keys[requested_key_index]; + auto it = attribute_container.find(requested_key); if (it != attribute_container.end()) { - auto item = it->second; - result_data.emplace_back(item); + ValueType item = getValueFromCell(it); + result_data[requested_key_index] = item; } else { - row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); - result_data.emplace_back(row_value_to_insert.get>()); + auto default_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); + result_data[requested_key_index] = default_value_to_insert.get>(); } } } @@ -163,27 +266,25 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co const auto & keys = extractor.getKeys(); size_t keys_size = keys.size(); - auto result = ColumnUInt8::create(keys_size); + auto result = ColumnUInt8::create(keys_size, false); auto& out = result->getData(); - const auto & attribute = attributes.front(); - - auto type_call = [&](const auto & dictionary_attribute_type) + if (attributes.empty()) { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; + query_count.fetch_add(keys_size, std::memory_order_relaxed); + return result; + } - const auto & attribute_map = std::get>(attribute.container); + /// Contaiiner + getAttributeContainer(0, [&](const auto & container) + { for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) { const auto & requested_key = keys[requested_key_index]; - out[requested_key_index] = attribute_map.find(requested_key) != attribute_map.end(); + out[requested_key_index] = container.find(requested_key) != container.end(); } - }; - - callOnDictionaryAttributeType(attribute.type, type_call); + }); query_count.fetch_add(keys_size, std::memory_order_relaxed); @@ -208,7 +309,10 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr auto is_key_valid_func = [&](auto & key) { - return parent_keys_map.find(key) != parent_keys_map.end(); + if constexpr (sparse) + return parent_keys_map.find(key) != parent_keys_map.end(); + else + return parent_keys_map.find(key) != nullptr; }; auto get_parent_func = [&](auto & hierarchy_key) @@ -217,10 +321,16 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr auto it = parent_keys_map.find(hierarchy_key); - if (it == parent_keys_map.end()) - return result; - - result = it->second; + if constexpr (sparse) + { + if (it != parent_keys_map.end()) + result = it->second; + } + else + { + if (it != nullptr) + result = it->getMapped(); + } return result; }; @@ -259,7 +369,10 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( auto is_key_valid_func = [&](auto & key) { - return parent_keys_map.find(key) != parent_keys_map.end(); + if constexpr (sparse) + return parent_keys_map.find(key) != parent_keys_map.end(); + else + return parent_keys_map.find(key) != nullptr; }; auto get_parent_func = [&](auto & hierarchy_key) @@ -268,10 +381,16 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( auto it = parent_keys_map.find(hierarchy_key); - if (it == parent_keys_map.end()) - return result; - - result = it->second; + if constexpr (sparse) + { + if (it != parent_keys_map.end()) + result = it->second; + } + else + { + if (it != nullptr) + result = it->getMapped(); + } return result; }; @@ -335,7 +454,7 @@ void HashedDictionary::createAttributes() if (is_complex_type) { - Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, CollectionType(), std::move(string_arena)}; + Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, ComplexAttributeCollectionType(), std::move(string_arena)}; attributes.emplace_back(std::move(attribute)); } else @@ -573,9 +692,25 @@ void HashedDictionary::calculateBytesAllocated() { getAttributeContainer(i, [&](const auto & container) { - /// TODO: Calculate + using ContainerType = std::decay_t; + using AttributeValueType = typename ContainerType::mapped_type; + bytes_allocated += sizeof(container); + + if constexpr (sparse || std::is_same_v) + { + bytes_allocated += container.max_size() * (sizeof(KeyType) + sizeof(AttributeValueType)); + bucket_count = container.bucket_count(); + } + else + { + bytes_allocated += container.getBufferSizeInBytes(); + bucket_count = container.getBufferSizeInCells(); + } }); + + if (attributes[i].string_arena) + bytes_allocated += attributes[i].string_arena->size(); } bytes_allocated += complex_key_arena.size(); @@ -614,7 +749,7 @@ void HashedDictionary::getAttributeContainer(size_t if (unlikely(attribute.is_complex_type)) { - auto & attribute_container = std::get>(attribute.container); + auto & attribute_container = std::get>(attribute.container); std::forward(get_container_func)(attribute_container); } else diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index febab94b255..3e7207e19cf 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -111,7 +111,16 @@ public: private: template - using CollectionTypeNonSparse = std::conditional_t, absl::flat_hash_map>>; + using CollectionTypeNonSparse = std::conditional_t< + dictionary_key_type == DictionaryKeyType::simple, + HashMap, + HashMapWithSavedHash>>; + + template + using ComplexAttributeCollectionTypeNonSparse = std::conditional_t< + dictionary_key_type == DictionaryKeyType::simple, + absl::flat_hash_map>, + absl::flat_hash_map>>; #if !defined(ARCADIA_BUILD) template @@ -122,11 +131,20 @@ private: #endif template - using CollectionTypeSparse = std::conditional_t, SparseHashMap>; + using CollectionTypeSparse = std::conditional_t< + dictionary_key_type == DictionaryKeyType::simple, + SparseHashMap, + SparseHashMap>; template using CollectionType = std::conditional_t, CollectionTypeNonSparse>; + template + using ComplexAttributeCollectionType = std::conditional_t< + sparse, + CollectionTypeSparse, + ComplexAttributeCollectionTypeNonSparse>; + struct Attribute final { AttributeUnderlyingType type; @@ -147,7 +165,7 @@ private: CollectionType, CollectionType, CollectionType, - CollectionType> + ComplexAttributeCollectionType> container; std::unique_ptr string_arena; }; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index 8839e383843..c0cc0f2c5df 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -70,6 +70,7 @@ namespace ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type) { + key_column = key_column->convertToFullColumnIfConst(); const auto * key_column_typed = checkAndGetColumn>(*key_column); if (!key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); @@ -106,6 +107,9 @@ ColumnUInt8::Ptr isInHierarchyDefaultImplementation( ColumnPtr in_key_column, const DataTypePtr & key_type) { + key_column = key_column->convertToFullColumnIfConst(); + in_key_column = in_key_column->convertToFullColumnIfConst(); + const auto * key_column_typed = checkAndGetColumn>(*key_column); if (!key_column_typed) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); From 13417b5b40e01ad394c22d837df8c1e9ceb5d74a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Mar 2021 00:47:43 +0300 Subject: [PATCH 081/266] Added documentation --- src/Common/HashTable/HashMap.h | 6 +- src/Dictionaries/CacheDictionary.cpp | 4 +- src/Dictionaries/DirectDictionary.cpp | 4 +- src/Dictionaries/FlatDictionary.cpp | 7 +- src/Dictionaries/HashedDictionary.cpp | 10 +- .../HierarchyDictionariesUtils.cpp | 34 ++-- src/Dictionaries/HierarchyDictionariesUtils.h | 190 +++++++++++++----- 7 files changed, 179 insertions(+), 76 deletions(-) diff --git a/src/Common/HashTable/HashMap.h b/src/Common/HashTable/HashMap.h index cfa159cfdc6..c3cd09eccb2 100644 --- a/src/Common/HashTable/HashMap.h +++ b/src/Common/HashTable/HashMap.h @@ -134,7 +134,8 @@ struct HashMapCell }; -namespace std { +namespace std +{ template struct tuple_size> : std::integral_constant { }; @@ -257,7 +258,8 @@ public: } }; -namespace std { +namespace std +{ template struct tuple_size> : std::integral_constant { }; diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 26f44dd42c5..f1abe98b454 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -374,7 +374,7 @@ ColumnPtr CacheDictionary::getHierarchy( { if (dictionary_key_type == DictionaryKeyType::simple) { - auto result = getHierarchyDefaultImplementation(this, key_column, key_type); + auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type); query_count.fetch_add(key_column->size(), std::memory_order_relaxed); return result; } @@ -390,7 +390,7 @@ ColumnUInt8::Ptr CacheDictionary::isInHierarchy( { if (dictionary_key_type == DictionaryKeyType::simple) { - auto result = isInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); + auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); query_count.fetch_add(key_column->size(), std::memory_order_relaxed); return result; } diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 5f03dd44ee7..4ee6d3c346d 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -192,7 +192,7 @@ ColumnPtr DirectDictionary::getHierarchy( { if (dictionary_key_type == DictionaryKeyType::simple) { - auto result = getHierarchyDefaultImplementation(this, key_column, key_type); + auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type); query_count.fetch_add(key_column->size(), std::memory_order_relaxed); return result; } @@ -208,7 +208,7 @@ ColumnUInt8::Ptr DirectDictionary::isInHierarchy( { if (dictionary_key_type == DictionaryKeyType::simple) { - auto result = isInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); + auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type); query_count.fetch_add(key_column->size(), std::memory_order_relaxed); return result; } diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index ce9b2792688..2d8d208d76b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -213,10 +213,7 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy( return result; }; - auto is_in_hierarchy_result = isInKeysHierarchy(keys, keys_in, null_value, is_key_valid_func, get_parent_key_func); - - auto result = ColumnUInt8::create(); - result->getData() = std::move(is_in_hierarchy_result); + auto result = getKeysIsInHierarchyColumn(keys, keys_in, null_value, is_key_valid_func, get_parent_key_func); query_count.fetch_add(keys.size(), std::memory_order_relaxed); @@ -245,7 +242,7 @@ ColumnPtr FlatDictionary::getDescendants( parent_to_child[parent_key].emplace_back(static_cast(i)); } - auto result = getDescendantsArray(keys, parent_to_child, level); + auto result = getKeysDescendantsArray(keys, parent_to_child, level); query_count.fetch_add(keys.size(), std::memory_order_relaxed); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 7ebc43e1a72..0f0c45f9e56 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -395,10 +395,9 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( return result; }; - auto is_in_hierarchy_result = isInKeysHierarchy(keys, keys_in, null_value, is_key_valid_func, get_parent_func); + auto result = getKeysIsInHierarchyColumn(keys, keys_in, null_value, is_key_valid_func, get_parent_func); - auto result = ColumnUInt8::create(); - result->getData() = std::move(is_in_hierarchy_result); + query_count.fetch_add(keys.size(), std::memory_order_relaxed); return result; } @@ -427,7 +426,10 @@ ColumnPtr HashedDictionary::getDescendants( for (const auto & [key, value] : parent_keys) parent_to_child[value].emplace_back(key); - auto result = getDescendantsArray(keys, parent_to_child, level); + auto result = getKeysDescendantsArray(keys, parent_to_child, level); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + return result; } else diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index c0cc0f2c5df..fdf13bc6d1c 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -10,7 +10,17 @@ namespace ErrorCodes namespace { - HashMap getHierarchyMapImpl(const IDictionary * dictionary, const DictionaryAttribute & dictionary_attribute, const PaddedPODArray & initial_keys_to_request, const DataTypePtr & key_type) + /** In case of cache or direct dictionary we does not have structure with child to parent representation. + * This function build such structure calling getColumn for initial keys to request and for next keys in hierarchy, + * until all keys are requested or result key is null value. + * To distinquish null value key and key that is not present in dictionary, we use special default value column + * with max UInt64 value, if result column key has such value we assume that current key is not presented in dictionary storage. + */ + HashMap getChildToParentHierarchyMapImpl( + const IDictionary * dictionary, + const DictionaryAttribute & dictionary_attribute, + const PaddedPODArray & initial_keys_to_request, + const DataTypePtr & key_type) { UInt64 null_value = dictionary_attribute.null_value.get(); @@ -26,11 +36,11 @@ namespace PaddedPODArray next_keys_to_request; HashSet already_requested_keys; - HashMap key_to_parent_key; + HashMap child_to_parent_key; while (!keys_to_request.empty()) { - key_to_parent_key.reserve(key_to_parent_key.size() + keys_to_request.size()); + child_to_parent_key.reserve(child_to_parent_key.size() + keys_to_request.size()); auto parent_key_column = dictionary->getColumn(dictionary_attribute.name, dictionary_attribute.type, {key_to_request_column}, {key_type}, key_not_in_storage_default_value_column); @@ -50,7 +60,7 @@ namespace if (parent_key == key_not_in_storage_value) continue; - key_to_parent_key[key] = parent_key; + child_to_parent_key[key] = parent_key; if (parent_key == null_value || already_requested_keys.find(parent_key) != nullptr) @@ -64,11 +74,11 @@ namespace keys_to_request.assign(next_keys_to_request); } - return key_to_parent_key; + return child_to_parent_key; } } -ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type) +ColumnPtr getKeysHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type) { key_column = key_column->convertToFullColumnIfConst(); const auto * key_column_typed = checkAndGetColumn>(*key_column); @@ -79,7 +89,7 @@ ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, Colu const auto & dictionary_attribute = dictionary_structure.attributes[0]; const PaddedPODArray & requested_keys = key_column_typed->getData(); - HashMap key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; @@ -101,7 +111,7 @@ ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, Colu return dictionary_hierarchy_array; } -ColumnUInt8::Ptr isInHierarchyDefaultImplementation( +ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( const IDictionary * dictionary, ColumnPtr key_column, ColumnPtr in_key_column, @@ -122,7 +132,7 @@ ColumnUInt8::Ptr isInHierarchyDefaultImplementation( const auto & dictionary_attribute = dictionary_structure.attributes[0]; const PaddedPODArray & requested_keys = key_column_typed->getData(); - HashMap key_to_parent_key = getHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; @@ -141,11 +151,7 @@ ColumnUInt8::Ptr isInHierarchyDefaultImplementation( UInt64 null_value = dictionary_attribute.null_value.get(); const auto & in_keys = in_key_column_typed->getData(); - auto is_in_hierarchy_result = isInKeysHierarchy(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func); - - auto result = ColumnUInt8::create(); - result->getData() = std::move(is_in_hierarchy_result); - + auto result = getKeysIsInHierarchyColumn(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func); return result; } diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 0589662e0bb..8cdcb5f43f2 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -35,14 +35,33 @@ namespace detail std::optional operator()(T key [[maybe_unused]]) { return {}; } }; + /** Calculate hierarchy for keys iterating the hierarchy from child to parent using get_parent_key_func provided by client. + * Hierarchy iteration is stopped if key equals null value, get_parent_key_func returns null optional, or hierarchy depth + * greater or equal than DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH. + * IsKeyValidFunc used for each input hierarchy key, if it returns false result hierarchy for that key will have size 0. + * Hierarchy result is ElementsAndOffsets structure, for each element there is hierarchy array, + * with size offset[element_index] - (element_index > 0 ? offset[element_index - 1] : 0). + * + * Example: + * id parent_id + * 1 0 + * 2 1 + * 3 1 + * 4 2 + * + * If hierarchy_null_value will be 0. Requested keys [1, 2, 3, 4, 5]. + * Result: [1], [2, 1], [3, 1], [4, 2, 1], [] + * Elements: [1, 2, 1, 3, 4, 4, 2, 1] + * Offsets: [1, 2, 2, 3, 0] + */ template - ElementsAndOffsets getKeysHierarchy( - const PaddedPODArray & hierarchy_keys, + ElementsAndOffsets getHierarchy( + const PaddedPODArray & keys, const KeyType & hierarchy_null_value, IsKeyValidFunc && is_key_valid_func, - GetParentKeyFunc && get_key_func) + GetParentKeyFunc && get_parent_key_func) { - size_t hierarchy_keys_size = hierarchy_keys.size(); + size_t hierarchy_keys_size = keys.size(); PaddedPODArray elements; elements.reserve(hierarchy_keys_size); @@ -61,7 +80,7 @@ namespace detail for (size_t i = 0; i < hierarchy_keys_size; ++i) { - auto hierarchy_key = hierarchy_keys[i]; + auto hierarchy_key = keys[i]; size_t current_hierarchy_depth = 0; bool is_key_valid = std::forward(is_key_valid_func)(hierarchy_key); @@ -112,7 +131,7 @@ namespace detail elements.emplace_back(hierarchy_key); ++current_hierarchy_depth; - std::optional parent_key = std::forward(get_key_func)(hierarchy_key); + std::optional parent_key = std::forward(get_parent_key_func)(hierarchy_key); if (!parent_key.has_value()) break; @@ -128,20 +147,91 @@ namespace detail return result; } + /** Returns array with UInt8 represent if key from in_keys array is in hierarchy of key from keys column. + * If value in result array is 1 that means key from in_keys array is in hierarchy of key from + * keys array with same index, 0 therwise. + * For getting hierarchy implementation uses getKeysHierarchy function. + * + * Not: keys size must be equal to in_keys_size. + */ + template + PaddedPODArray getIsInHierarchy( + const PaddedPODArray & keys, + const PaddedPODArray & in_keys, + const KeyType & hierarchy_null_value, + IsKeyValidFunc && is_key_valid_func, + GetParentKeyFunc && get_parent_func) + { + assert(hierarchy_keys.size() == hierarchy_in_keys.size()); + + PaddedPODArray result; + result.resize_fill(keys.size()); + + detail::ElementsAndOffsets hierarchy = detail::getHierarchy( + keys, + hierarchy_null_value, + std::forward(is_key_valid_func), + std::forward(get_parent_func)); + + auto & offsets = hierarchy.offsets; + auto & elements = hierarchy.elements; + + for (size_t i = 0; i < offsets.size(); ++i) + { + size_t i_elements_start = i > 0 ? offsets[i - 1] : 0; + size_t i_elements_end = offsets[i]; + + auto & key_to_find = in_keys[i]; + + const auto * begin = elements.begin() + i_elements_start; + const auto * end = elements.begin() + i_elements_end; + + const auto * it = std::find(begin, end, key_to_find); + + bool contains_key = (it != end); + result[i] = contains_key; + } + + return result; + } + struct GetAllDescendantsStrategy { size_t level = 0; }; struct GetDescendantsAtSpecificLevelStrategy { size_t level = 0; }; + /** Get descendants for keys iterating the hierarchy from parent to child using parent_to_child hash map provided by client. + * GetAllDescendantsStrategy get all descendants for key + * GetDescendantsAtSpecificLevelStrategy get descendants only for specific hierarchy level. + * Hierarchy result is ElementsAndOffsets structure, for each element there is descendants array, + * with size offset[element_index] - (element_index > 0 ? offset[element_index - 1] : 0). + * + * Example: + * id parent_id + * 1 0 + * 2 1 + * 3 1 + * 4 2 + * + * Example. Strategy GetAllDescendantsStrategy. + * Requested keys [0, 1, 2, 5]. + * Result: [2, 3, 4], [2, 4], [4], [] + * Elements: [2, 3, 4, 2, 4, 4] + * Offsets: [3, 2, 1, 0] + */ template ElementsAndOffsets getDescendants( - const PaddedPODArray & requested_keys, + const PaddedPODArray & keys, const HashMap> & parent_to_child, Strategy strategy) { + /// If strategy is GetAllDescendantsStrategy we try to cache and later reuse previously calculated descendants. + /// If strategy is GetDescendantsAtSpecificLevelStrategy we does not use cache strategy. + size_t keys_size = keys.size(); + PaddedPODArray descendants; - descendants.reserve(requested_keys.size()); + descendants.reserve(keys_size); PaddedPODArray descendants_offsets; - descendants_offsets.reserve(requested_keys.size()); + descendants_offsets.reserve(keys_size); struct Range { @@ -153,7 +243,7 @@ namespace detail HashMap already_processed_keys_to_range [[maybe_unused]]; if constexpr (std::is_same_v) - already_processed_keys_to_range.reserve(requested_keys.size()); + already_processed_keys_to_range.reserve(keys_size); struct KeyAndDepth { @@ -162,16 +252,16 @@ namespace detail }; HashSet already_processed_keys_during_loop; - already_processed_keys_during_loop.reserve(requested_keys.size()); + already_processed_keys_during_loop.reserve(keys_size); PaddedPODArray next_keys_to_process_stack; - next_keys_to_process_stack.reserve(requested_keys.size()); + next_keys_to_process_stack.reserve(keys_size); Int64 level = static_cast(strategy.level); - for (size_t i = 0; i < requested_keys.size(); ++i) + for (size_t i = 0; i < keys_size; ++i) { - const KeyType & requested_key = requested_keys[i]; + const KeyType & requested_key = keys[i]; if (parent_to_child.find(requested_key) == nullptr) { @@ -181,6 +271,11 @@ namespace detail next_keys_to_process_stack.emplace_back(KeyAndDepth{requested_key, 0}); + /** To cache range for key without recursive function calls and custom stack we put special + * signaling value on stack key_range_requires_update. + * When we pop such value from stack that means processing descendants for key is finished + * and we can update range with end_index. + */ while (!next_keys_to_process_stack.empty()) { KeyAndDepth key_to_process = next_keys_to_process_stack.back(); @@ -191,6 +286,7 @@ namespace detail if constexpr (std::is_same_v) { + /// Update end_index for key if (depth == key_range_requires_update) { auto * it = already_processed_keys_to_range.find(key); @@ -248,19 +344,22 @@ namespace detail if constexpr (std::is_same_v) { + /// Put special signaling value on stack and update cache with range start size_t range_start_index = descendants.size(); already_processed_keys_to_range[key].start_index = range_start_index; - next_keys_to_process_stack.emplace_back(KeyAndDepth{key, -1}); + next_keys_to_process_stack.emplace_back(KeyAndDepth{key, key_range_requires_update}); } already_processed_keys_during_loop.insert(key); ++depth; - const auto & childs = it->getMapped(); + const auto & children = it->getMapped(); - for (auto child_key : childs) + for (auto child_key : children) { + /// In case of GetAllDescendantsStrategy we add any descendant to result array + /// If strategy is GetDescendantsAtSpecificLevelStrategy we require depth == level if (std::is_same_v || depth == level) descendants.emplace_back(child_key); @@ -277,6 +376,7 @@ namespace detail return result; } + /// Converts ElementAndOffsets structure into ArrayColumn template ColumnPtr convertElementsAndOffsetsIntoArray(ElementsAndOffsets && elements_and_offsets) { @@ -292,60 +392,48 @@ namespace detail } } +/// Returns hierarchy array column for keys template ColumnPtr getKeysHierarchyArray( - const PaddedPODArray & hierarchy_keys, + const PaddedPODArray & keys, const KeyType & hierarchy_null_value, IsKeyValidFunc && is_key_valid_func, GetParentKeyFunc && get_parent_func) { - auto elements_and_offsets = detail::getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); + auto elements_and_offsets = detail::getHierarchy( + keys, + hierarchy_null_value, + std::forward(is_key_valid_func), + std::forward(get_parent_func)); + return detail::convertElementsAndOffsetsIntoArray(std::move(elements_and_offsets)); } +/// Returns is in hierarchy column for keys template -PaddedPODArray isInKeysHierarchy( +ColumnUInt8::Ptr getKeysIsInHierarchyColumn( const PaddedPODArray & hierarchy_keys, const PaddedPODArray & hierarchy_in_keys, const KeyType & hierarchy_null_value, IsKeyValidFunc && is_key_valid_func, GetParentKeyFunc && get_parent_func) { - assert(hierarchy_keys.size() == hierarchy_in_keys.size()); - - PaddedPODArray result; - result.resize_fill(hierarchy_keys.size()); - - detail::ElementsAndOffsets hierarchy = detail::getKeysHierarchy( + auto is_in_hierarchy_data = detail::getIsInHierarchy( hierarchy_keys, + hierarchy_in_keys, hierarchy_null_value, std::forward(is_key_valid_func), std::forward(get_parent_func)); - auto & offsets = hierarchy.offsets; - auto & elements = hierarchy.elements; - - for (size_t i = 0; i < offsets.size(); ++i) - { - size_t i_elements_start = i > 0 ? offsets[i - 1] : 0; - size_t i_elements_end = offsets[i]; - - auto & key_to_find = hierarchy_in_keys[i]; - - const auto * begin = elements.begin() + i_elements_start; - const auto * end = elements.begin() + i_elements_end; - - const auto * it = std::find(begin, end, key_to_find); - - bool contains_key = (it != end); - result[i] = contains_key; - } + auto result = ColumnUInt8::create(); + result->getData() = std::move(is_in_hierarchy_data); return result; } +/// Returns descendants array column for keys template -ColumnPtr getDescendantsArray( +ColumnPtr getKeysDescendantsArray( const PaddedPODArray & requested_keys, const HashMap> & parent_to_child, size_t level) @@ -364,12 +452,20 @@ ColumnPtr getDescendantsArray( } } -ColumnPtr getHierarchyDefaultImplementation( +/** Default getHierarchy implementation for dictionaries that does not have structure with child to parent representation. + * Implementation will build such structure with getColumn calls, and then getHierarchy for such structure. + * Returns ColumnArray with hierarchy arrays for keys from key_column. + */ +ColumnPtr getKeysHierarchyDefaultImplementation( const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type); -ColumnUInt8::Ptr isInHierarchyDefaultImplementation( +/** Default isInHierarchy implementation for dictionaries that does not have structure with child to parent representation. + * Implementation will build such structure with getColumn calls, and then getHierarchy for such structure. + * Returns UInt8 column if key from in_key_column is in key hierarchy from key_column. + */ +ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( const IDictionary * dictionary, ColumnPtr key_column, ColumnPtr in_key_column, From 2cac8d13cc25c105953d175552931cee9cf88523 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Mar 2021 16:57:21 +0300 Subject: [PATCH 082/266] Updated tests and documentation --- src/Common/PODArray.h | 23 ++ src/Common/tests/gtest_pod_array.cpp | 13 + src/Dictionaries/HashedDictionary.cpp | 40 +--- src/Dictionaries/HierarchyDictionariesUtils.h | 38 ++- ...ictionary_ssd_cache_dictionary_storage.cpp | 2 - .../gtest_hierarchy_dictionaries_utils.cpp | 225 ++++++++++++++++++ src/Functions/FunctionsExternalDictionaries.h | 133 +++++++---- .../01778_hierarchical_dictionaries.reference | 102 ++++++++ .../01778_hierarchical_dictionaries.sql | 95 ++++++++ 9 files changed, 571 insertions(+), 100 deletions(-) create mode 100644 src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp create mode 100644 tests/queries/0_stateless/01778_hierarchical_dictionaries.reference create mode 100644 tests/queries/0_stateless/01778_hierarchical_dictionaries.sql diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 57ad3d46177..3d4050fbffa 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -530,6 +530,29 @@ public: this->c_end += bytes_to_copy; } + template + void insertFromItself(iterator from_begin, iterator from_end, TAllocatorParams && ... allocator_params) + { + static_assert(memcpy_can_be_used_for_assignment, std::decay_t>); + + /// Convert iterators to indexes because reserve can invalidate iterators + size_t start_index = from_begin - begin(); + size_t end_index = from_end - begin(); + + assert(start_index <= end_index); + + size_t required_capacity = this->size() + (end_index - start_index); + if (required_capacity > this->capacity()) + this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); + + size_t bytes_to_copy = this->byte_size(end_index - start_index); + if (bytes_to_copy) + { + memcpy(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); + this->c_end += bytes_to_copy; + } + } + template void insert_assume_reserved(It1 from_begin, It2 from_end) { diff --git a/src/Common/tests/gtest_pod_array.cpp b/src/Common/tests/gtest_pod_array.cpp index 63cf7026757..9cc77b88195 100644 --- a/src/Common/tests/gtest_pod_array.cpp +++ b/src/Common/tests/gtest_pod_array.cpp @@ -33,6 +33,19 @@ TEST(Common, PODArrayInsert) EXPECT_EQ(str, std::string(chars.data(), chars.size())); } +TEST(Common, PODArrayInsertFromItself) +{ + { + PaddedPODArray array { 1 }; + + for (size_t i = 0; i < 3; ++i) + array.insertFromItself(array.begin(), array.end()); + + PaddedPODArray expected {1,1,1,1,1,1,1,1}; + ASSERT_EQ(array,expected); + } +} + TEST(Common, PODPushBackRawMany) { PODArray chars; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 0f0c45f9e56..40394370087 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -307,13 +307,7 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr const UInt64 null_value = dictionary_attribute.null_value.get(); const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); - auto is_key_valid_func = [&](auto & key) - { - if constexpr (sparse) - return parent_keys_map.find(key) != parent_keys_map.end(); - else - return parent_keys_map.find(key) != nullptr; - }; + auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); }; auto get_parent_func = [&](auto & hierarchy_key) { @@ -321,16 +315,8 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr auto it = parent_keys_map.find(hierarchy_key); - if constexpr (sparse) - { - if (it != parent_keys_map.end()) - result = it->second; - } - else - { - if (it != nullptr) - result = it->getMapped(); - } + if (it != parent_keys_map.end()) + result = getValueFromCell(it); return result; }; @@ -367,13 +353,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( const UInt64 null_value = dictionary_attribute.null_value.get(); const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); - auto is_key_valid_func = [&](auto & key) - { - if constexpr (sparse) - return parent_keys_map.find(key) != parent_keys_map.end(); - else - return parent_keys_map.find(key) != nullptr; - }; + auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); }; auto get_parent_func = [&](auto & hierarchy_key) { @@ -381,16 +361,8 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( auto it = parent_keys_map.find(hierarchy_key); - if constexpr (sparse) - { - if (it != parent_keys_map.end()) - result = it->second; - } - else - { - if (it != nullptr) - result = it->getMapped(); - } + if (it != parent_keys_map.end()) + result = getValueFromCell(it); return result; }; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 8cdcb5f43f2..f8a1252f1b2 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -51,8 +51,8 @@ namespace detail * * If hierarchy_null_value will be 0. Requested keys [1, 2, 3, 4, 5]. * Result: [1], [2, 1], [3, 1], [4, 2, 1], [] - * Elements: [1, 2, 1, 3, 4, 4, 2, 1] - * Offsets: [1, 2, 2, 3, 0] + * Elements: [1, 2, 1, 3, 1, 4, 2, 1] + * Offsets: [1, 3, 5, 8, 8] */ template ElementsAndOffsets getHierarchy( @@ -113,14 +113,7 @@ namespace detail size_t end_index = offsets[offset]; current_hierarchy_depth += end_index - start_index; - - /// TODO: Insert part of pod array into itself - while (start_index < end_index) - { - elements.emplace_back(elements[start_index]); - ++start_index; - } - + elements.insertFromItself(elements.begin() + start_index, elements.begin() + end_index); break; } @@ -162,7 +155,7 @@ namespace detail IsKeyValidFunc && is_key_valid_func, GetParentKeyFunc && get_parent_func) { - assert(hierarchy_keys.size() == hierarchy_in_keys.size()); + assert(keys.size() == in_keys.size()); PaddedPODArray result; result.resize_fill(keys.size()); @@ -212,10 +205,15 @@ namespace detail * 4 2 * * Example. Strategy GetAllDescendantsStrategy. - * Requested keys [0, 1, 2, 5]. - * Result: [2, 3, 4], [2, 4], [4], [] - * Elements: [2, 3, 4, 2, 4, 4] - * Offsets: [3, 2, 1, 0] + * Requested keys [0, 1, 2, 3, 4]. + * Result: [1, 2, 3, 4], [2, 2, 4], [4], [], [] + * Elements: [1, 2, 3, 4, 2, 3, 4, 4] + * Offsets: [4, 7, 8, 8, 8] + * + * Example. Strategy GetDescendantsAtSpecificLevelStrategy with level 1. + * Requested keys [0, 1, 2, 3, 4]. + * Result: [1], [2, 3], [4], [], []; + * Offsets: [1, 3, 4, 4, 4]; */ template ElementsAndOffsets getDescendants( @@ -319,13 +317,9 @@ namespace detail } else { - /// TODO: Insert part of pod array - while (range.start_index != range.end_index) - { - descendants.emplace_back(descendants[range.start_index]); - ++range.start_index; - } - + auto insert_start_iterator = descendants.begin() + range.start_index; + auto insert_end_iterator = descendants.begin() + range.end_index; + descendants.insertFromItself(insert_start_iterator, insert_end_iterator); continue; } } diff --git a/src/Dictionaries/tests/gtest_dictionary_ssd_cache_dictionary_storage.cpp b/src/Dictionaries/tests/gtest_dictionary_ssd_cache_dictionary_storage.cpp index 20529e91bd3..9fd9dc9b78c 100644 --- a/src/Dictionaries/tests/gtest_dictionary_ssd_cache_dictionary_storage.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_ssd_cache_dictionary_storage.cpp @@ -1,7 +1,5 @@ #if defined(__linux__) || defined(__FreeBSD__) -#include - #include #include diff --git a/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp b/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp new file mode 100644 index 00000000000..064f57dfe11 --- /dev/null +++ b/src/Dictionaries/tests/gtest_hierarchy_dictionaries_utils.cpp @@ -0,0 +1,225 @@ +#include + +#include + +#include + +using namespace DB; + +TEST(HierarchyDictionariesUtils, getHierarchy) +{ + { + HashMap child_to_parent; + child_to_parent[1] = 0; + child_to_parent[2] = 1; + child_to_parent[3] = 1; + child_to_parent[4] = 2; + + auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; }; + + auto get_parent_key_func = [&](auto key) + { + auto it = child_to_parent.find(key); + std::optional value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); + return value; + }; + + UInt64 hierarchy_null_value_key = 0; + PaddedPODArray keys = {1, 2, 3, 4, 5}; + + auto result = DB::detail::getHierarchy( + keys, + hierarchy_null_value_key, + is_key_valid_func, + get_parent_key_func); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {1, 2, 1, 3, 1, 4, 2, 1}; + PaddedPODArray expected_offsets = {1, 3, 5, 8, 8}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } + { + HashMap child_to_parent; + child_to_parent[1] = 2; + child_to_parent[2] = 1; + + auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; }; + + auto get_parent_key_func = [&](auto key) + { + auto it = child_to_parent.find(key); + std::optional value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); + return value; + }; + + UInt64 hierarchy_null_value_key = 0; + PaddedPODArray keys = {1, 2, 3}; + + auto result = DB::detail::getHierarchy( + keys, + hierarchy_null_value_key, + is_key_valid_func, + get_parent_key_func); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {1, 2, 2}; + PaddedPODArray expected_offsets = {2, 3, 3}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } +} + +TEST(HierarchyDictionariesUtils, getIsInHierarchy) +{ + { + HashMap child_to_parent; + child_to_parent[1] = 0; + child_to_parent[2] = 1; + child_to_parent[3] = 1; + child_to_parent[4] = 2; + + auto is_key_valid_func = [&](auto key) { return child_to_parent.find(key) != nullptr; }; + + auto get_parent_key_func = [&](auto key) + { + auto it = child_to_parent.find(key); + std::optional value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); + return value; + }; + + UInt64 hierarchy_null_value_key = 0; + PaddedPODArray keys = {1, 2, 3, 4, 5}; + PaddedPODArray keys_in = {1, 1, 1, 2, 5}; + + PaddedPODArray actual = DB::detail::getIsInHierarchy( + keys, + keys_in, + hierarchy_null_value_key, + is_key_valid_func, + get_parent_key_func); + + PaddedPODArray expected = {1,1,1,1,0}; + + ASSERT_EQ(actual, expected); + } + { + HashMap child_to_parent; + child_to_parent[1] = 2; + child_to_parent[2] = 1; + + auto is_key_valid_func = [&](auto key) + { + return child_to_parent.find(key) != nullptr; + }; + + auto get_parent_key_func = [&](auto key) + { + auto it = child_to_parent.find(key); + std::optional value = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); + return value; + }; + + UInt64 hierarchy_null_value_key = 0; + PaddedPODArray keys = {1, 2, 3}; + PaddedPODArray keys_in = {1, 2, 3}; + + PaddedPODArray actual = DB::detail::getIsInHierarchy( + keys, + keys_in, + hierarchy_null_value_key, + is_key_valid_func, + get_parent_key_func); + + PaddedPODArray expected = {1, 1, 0}; + ASSERT_EQ(actual, expected); + } +} + +TEST(HierarchyDictionariesUtils, getDescendants) +{ + { + HashMap> parent_to_child; + parent_to_child[0].emplace_back(1); + parent_to_child[1].emplace_back(2); + parent_to_child[1].emplace_back(3); + parent_to_child[2].emplace_back(4); + + PaddedPODArray keys = {0, 1, 2, 3, 4}; + + { + auto result = DB::detail::getDescendants( + keys, + parent_to_child, + DB::detail::GetAllDescendantsStrategy()); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {1, 2, 3, 4, 2, 3, 4, 4}; + PaddedPODArray expected_offsets = {4, 7, 8, 8, 8}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } + { + auto result = DB::detail::getDescendants( + keys, + parent_to_child, + DB::detail::GetDescendantsAtSpecificLevelStrategy{1}); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {1, 2, 3, 4}; + PaddedPODArray expected_offsets = {1, 3, 4, 4, 4}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } + } + { + HashMap> parent_to_child; + parent_to_child[1].emplace_back(2); + parent_to_child[2].emplace_back(1); + + PaddedPODArray keys = {1, 2, 3}; + + { + auto result = DB::detail::getDescendants( + keys, + parent_to_child, + DB::detail::GetAllDescendantsStrategy()); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {2, 1, 1}; + PaddedPODArray expected_offsets = {2, 3, 3}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } + { + auto result = DB::detail::getDescendants( + keys, + parent_to_child, + DB::detail::GetDescendantsAtSpecificLevelStrategy{1}); + + const auto & actual_elements = result.elements; + const auto & actual_offsets = result.offsets; + + PaddedPODArray expected_elements = {2, 1}; + PaddedPODArray expected_offsets = {1, 2, 2}; + + ASSERT_EQ(actual_elements, expected_elements); + ASSERT_EQ(actual_offsets, expected_offsets); + } + } +} diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index b57ff369779..0bc22826f4e 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -79,9 +79,13 @@ public: return dict; } - std::shared_ptr getDictionary(const ColumnWithTypeAndName & column) + std::shared_ptr getDictionary(const ColumnPtr & column) { - const auto * dict_name_col = checkAndGetColumnConst(column.column.get()); + const auto * dict_name_col = checkAndGetColumnConst(column.get()); + + if (!dict_name_col) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected const String column"); + return getDictionary(dict_name_col->getValue()); } @@ -176,7 +180,7 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - auto dictionary = helper.getDictionary(arguments[0]); + auto dictionary = helper.getDictionary(arguments[0].column); auto dictionary_key_type = dictionary->getKeyType(); const ColumnWithTypeAndName & key_column_with_type = arguments[1]; @@ -716,12 +720,16 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) - throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() - + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of first argument of function ({}). Expected String. Actual type ({})", + getName(), + arguments[0]->getName()); if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})", + getName(), + arguments[1]->getName()); return std::make_shared(std::make_shared()); } @@ -733,7 +741,7 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - auto dictionary = helper.getDictionary(arguments[0]); + auto dictionary = helper.getDictionary(arguments[0].column); if (!dictionary->hasHierarchy()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, @@ -772,16 +780,22 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) - throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() - + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of first argument of function ({}). Expected String. Actual type ({})", + getName(), + arguments[0]->getName()); if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})", + getName(), + arguments[1]->getName()); if (!WhichDataType(arguments[2]).isUInt64()) - throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() - + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of third argument of function ({}). Expected UInt64. Actual type ({})", + getName(), + arguments[2]->getName()); return std::make_shared(); } @@ -793,7 +807,7 @@ private: if (input_rows_count == 0) return result_type->createColumn(); - auto dict = helper.getDictionary(arguments[0]); + auto dict = helper.getDictionary(arguments[0].column); if (!dict->hasHierarchy()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); @@ -826,35 +840,40 @@ private: bool useDefaultImplementationForConstants() const final { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } + bool isDeterministic() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (!isString(arguments[0])) - throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() - + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of first argument of function ({}). Expected String. Actual type ({})", + getName(), + arguments[0]->getName()); if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})", + getName(), + arguments[1]->getName()); return std::make_shared(std::make_shared()); } - bool isDeterministic() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { if (input_rows_count == 0) return result_type->createColumn(); - auto dict = helper.getDictionary(arguments[0]); + auto dictionary = helper.getDictionary(arguments[0].column); - if (!dict->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); + if (!dictionary->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Dictionary ({}) does not support hierarchy", + dictionary->getFullName()); - ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), 1); + ColumnPtr result = dictionary->getDescendants(arguments[1].column, std::make_shared(), 1); - return res; + return result; } mutable FunctionDictHelper helper; @@ -876,43 +895,73 @@ public: String getName() const override { return name; } private: - size_t getNumberOfArguments() const override { return 3; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const final { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 2}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; } + bool isDeterministic() const override { return false; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + size_t arguments_size = arguments.size(); + if (arguments_size < 2 || arguments_size > 3) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Illegal arguments size of function ({}). Expects 2 or 3 arguments size. Actual size ({})", + getName(), + arguments_size); + } + if (!isString(arguments[0])) - throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() - + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of first argument of function ({}). Expected const String. Actual type ({})", + getName(), + arguments[0]->getName()); if (!WhichDataType(arguments[1]).isUInt64()) - throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() - + ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of second argument of function ({}). Expected UInt64. Actual type ({})", + getName(), + arguments[1]->getName()); - if (!isUnsignedInteger(arguments[2])) - throw Exception{"Illegal type " + arguments[1]->getName() + " of third argument of function " + getName() - + ", must be const unsigned integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (arguments.size() == 3 && !isUnsignedInteger(arguments[2])) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of third argument of function ({}). Expected const unsigned integer. Actual type ({})", + getName(), + arguments[2]->getName()); + } return std::make_shared(std::make_shared()); } - bool isDeterministic() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { if (input_rows_count == 0) return result_type->createColumn(); - size_t level = static_cast(arguments[2].column->get64(0)); + auto dictionary = helper.getDictionary(arguments[0].column); - auto dict = helper.getDictionary(arguments[0]); + size_t level = 0; - if (!dict->hasHierarchy()) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName()); + if (arguments.size() == 3) + { + if (!isColumnConst(*arguments[2].column)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of third argument of function ({}). Expected const unsigned integer.", + getName()); - ColumnPtr res = dict->getDescendants(arguments[1].column, std::make_shared(), level); + level = static_cast(arguments[2].column->get64(0)); + } + + if (!dictionary->hasHierarchy()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Dictionary ({}) does not support hierarchy", + dictionary->getFullName()); + + ColumnPtr res = dictionary->getDescendants(arguments[1].column, std::make_shared(), level); return res; } diff --git a/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference b/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference new file mode 100644 index 00000000000..48245356ced --- /dev/null +++ b/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference @@ -0,0 +1,102 @@ +Flat dictionary +Get hierarchy +[] +[1] +[2,1] +[3,1] +[4,2,1] +[] +Get is in hierarchy +0 +1 +1 +1 +1 +0 +Get children +[1] +[2,3] +[4] +[] +[] +[] +Get all descendants +[1,2,3,4] +[2,3,4] +[4] +[] +[] +[] +Get descendants at first level +[1] +[2,3] +[4] +[] +[] +[] +Hashed dictionary +Get hierarchy +[] +[1] +[2,1] +[3,1] +[4,2,1] +[] +Get is in hierarchy +0 +1 +1 +1 +1 +0 +Get children +[1] +[2,3] +[4] +[] +[] +[] +Get all descendants +[1,2,3,4] +[2,3,4] +[4] +[] +[] +[] +Get descendants at first level +[1] +[2,3] +[4] +[] +[] +[] +Cache dictionary +Get hierarchy +[] +[1] +[2,1] +[3,1] +[4,2,1] +[] +Get is in hierarchy +0 +1 +1 +1 +1 +0 +Direct dictionary +Get hierarchy +[] +[1] +[2,1] +[3,1] +[4,2,1] +[] +Get is in hierarchy +0 +1 +1 +1 +1 +0 diff --git a/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql b/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql new file mode 100644 index 00000000000..eaf609e50fa --- /dev/null +++ b/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql @@ -0,0 +1,95 @@ +DROP DATABASE IF EXISTS 01778_db; +CREATE DATABASE 01778_db; + +CREATE TABLE 01778_db.simple_key_hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog; +INSERT INTO 01778_db.simple_key_hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2); + +CREATE DICTIONARY 01778_db.simple_key_hierarchy_flat_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +LAYOUT(FLAT()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Flat dictionary'; + +SELECT 'Get hierarchy'; +SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get is in hierarchy'; +SELECT dictIsIn('01778_db.simple_key_hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT 'Get children'; +SELECT dictGetChildren('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get all descendants'; +SELECT dictGetDescendants('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get descendants at first level'; +SELECT dictGetDescendants('01778_db.simple_key_hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6; + +DROP DICTIONARY 01778_db.simple_key_hierarchy_flat_dictionary; + +CREATE DICTIONARY 01778_db.simple_key_hierarchy_hashed_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +LAYOUT(FLAT()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Hashed dictionary'; + +SELECT 'Get hierarchy'; +SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get is in hierarchy'; +SELECT dictIsIn('01778_db.simple_key_hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT 'Get children'; +SELECT dictGetChildren('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get all descendants'; +SELECT dictGetDescendants('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get descendants at first level'; +SELECT dictGetDescendants('01778_db.simple_key_hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6; + +DROP DICTIONARY 01778_db.simple_key_hierarchy_hashed_dictionary; + +CREATE DICTIONARY 01778_db.simple_key_hierarchy_cache_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +LAYOUT(CACHE(SIZE_IN_CELLS 10)) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Cache dictionary'; + +SELECT 'Get hierarchy'; +SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get is in hierarchy'; +SELECT dictIsIn('01778_db.simple_key_hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6; + +DROP DICTIONARY 01778_db.simple_key_hierarchy_cache_dictionary; + +CREATE DICTIONARY 01778_db.simple_key_hierarchy_direct_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +LAYOUT(DIRECT()); + +SELECT 'Direct dictionary'; + +SELECT 'Get hierarchy'; +SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6; +SELECT 'Get is in hierarchy'; +SELECT dictIsIn('01778_db.simple_key_hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6; + +DROP DICTIONARY 01778_db.simple_key_hierarchy_direct_dictionary; + +DROP TABLE 01778_db.simple_key_hierarchy_source_table; +DROP DATABASE 01778_db; From 5fd575977af66fa43de055f67f3c16cc4ae74055 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Mar 2021 18:42:04 +0300 Subject: [PATCH 083/266] Fixed typos --- src/Dictionaries/HierarchyDictionariesUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index fdf13bc6d1c..99ff7aa63fa 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -13,7 +13,7 @@ namespace /** In case of cache or direct dictionary we does not have structure with child to parent representation. * This function build such structure calling getColumn for initial keys to request and for next keys in hierarchy, * until all keys are requested or result key is null value. - * To distinquish null value key and key that is not present in dictionary, we use special default value column + * To distinguish null value key and key that is not present in dictionary, we use special default value column * with max UInt64 value, if result column key has such value we assume that current key is not presented in dictionary storage. */ HashMap getChildToParentHierarchyMapImpl( From 0dfd091011442df5a629a94e6ee372e01cccd066 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Mar 2021 19:40:02 +0300 Subject: [PATCH 084/266] less flaky functional tests --- tests/clickhouse-test | 6 ++-- ...4_performance_introspection_and_logging.sh | 2 +- .../00956_sensitive_data_masking.sh | 2 +- .../01231_log_queries_min_type.sql | 9 +++--- tests/queries/0_stateless/01304_direct_io.sh | 10 +++--- .../01344_min_bytes_to_use_mmap_io_index.sql | 2 +- ...terialized_view_with_join_on_query_log.sql | 4 +-- .../0_stateless/01475_read_subcolumns.sql | 8 ++--- .../01531_query_log_query_comment.sql | 4 +-- .../0_stateless/01533_multiple_nested.sql | 4 +-- ...1546_log_queries_min_query_duration_ms.sql | 12 +++---- .../01547_query_log_current_database.sql | 3 +- .../01548_query_log_query_execution_ms.sql | 3 +- .../0_stateless/01600_benchmark_query.sh | 9 +++--- ...01686_event_time_microseconds_part_log.sql | 2 ++ .../0_stateless/01702_system_query_log.sql | 2 +- tests/queries/skip_list.json | 31 ++++++++++--------- 17 files changed, 55 insertions(+), 58 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 908da85c1c1..1604c08dcb6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -170,7 +170,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std if need_drop_database: clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) - seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10) + seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) try: clickhouse_proc_create.communicate(("DROP DATABASE " + database), timeout=seconds_left) except TimeoutExpired: @@ -207,7 +207,7 @@ def need_retry(stderr): def get_processlist(args): try: clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10) + (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=20) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) @@ -352,7 +352,7 @@ def run_tests_array(all_tests_with_params): clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) failed_to_check = False try: - clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=10) + clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)), timeout=20) except: failed_to_check = True diff --git a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh index e51e4fea5db..cc5ece15435 100755 --- a/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh +++ b/tests/queries/0_stateless/00634_performance_introspection_and_logging.sh @@ -48,7 +48,7 @@ SELECT threads_realtime >= threads_time_user_system_io, any(length(thread_ids)) >= 1 FROM - (SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1) + (SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-2 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1) ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV" # Clean diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index 764cb6a713e..6b9990ee819 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -97,7 +97,7 @@ echo 7 # and finally querylog $CLICKHOUSE_CLIENT \ --server_logs_file=/dev/null \ - --query="select * from system.query_log where current_database = currentDatabase() AND event_time > now() - 10 and query like '%TOPSECRET%';" + --query="select * from system.query_log where current_database = currentDatabase() AND event_date >= yesterday() and query like '%TOPSECRET%';" rm -f "$tmp_file" >/dev/null 2>&1 diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 9659739b61d..382b7f0bf7e 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -2,18 +2,18 @@ set log_queries=1; select '01231_log_queries_min_type/QUERY_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date >= yesterday(); set log_queries_min_type='EXCEPTION_BEFORE_START'; select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date >= yesterday(); set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing'; +select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; @@ -21,7 +21,6 @@ select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and - event_date = today() and - event_time >= now() - interval 1 minute and + event_date >= yesterday() and type = 'ExceptionWhileProcessing' and has(Settings.Names, 'max_rows_to_read'); diff --git a/tests/queries/0_stateless/01304_direct_io.sh b/tests/queries/0_stateless/01304_direct_io.sh index 3ba3d020d99..7505173ddba 100755 --- a/tests/queries/0_stateless/01304_direct_io.sh +++ b/tests/queries/0_stateless/01304_direct_io.sh @@ -9,12 +9,12 @@ $CLICKHOUSE_CLIENT --multiquery --query " CREATE TABLE bug (UserID UInt64, Date Date) ENGINE = MergeTree ORDER BY Date; INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000); OPTIMIZE TABLE bug FINAL;" +LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$LOG" +cat "$LOG" | grep Exception +cat "$LOG" | grep Loaded -$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err -cat "$CLICKHOUSE_TMP"/err | grep Exception -cat "$CLICKHOUSE_TMP"/err | grep Loaded - -rm "$CLICKHOUSE_TMP"/err +rm "$LOG" $CLICKHOUSE_CLIENT --multiquery --query " DROP TABLE bug;" diff --git a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql index 9044ee08f8d..7aab991d203 100644 --- a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql +++ b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql @@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1; SELECT * FROM test_01344 WHERE x = 'Hello, world'; SYSTEM FLUSH LOGS; -SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1; +SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1; DROP TABLE test_01344; diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index 950d4fe097f..3380f04f8c9 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -17,7 +17,7 @@ CREATE MATERIALIZED VIEW slow_log Engine=Memory AS extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, * FROM system.query_log - WHERE type<>1 and event_date >= yesterday() and event_time > now() - 120 + WHERE type<>1 and event_date >= yesterday() ) as ql INNER JOIN expected_times USING (QUERY_GROUP_ID) WHERE query_duration_ms > max_query_duration_ms @@ -38,7 +38,7 @@ SELECT extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, count() FROM system.query_log -WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>'' +WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and QUERY_GROUP_ID<>'' GROUP BY QUERY_GROUP_ID ORDER BY QUERY_GROUP_ID; diff --git a/tests/queries/0_stateless/01475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql index 16832c4fc59..3457d17dba1 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -10,7 +10,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====tuple===='; DROP TABLE IF EXISTS t_tup; @@ -27,7 +27,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====nullable===='; DROP TABLE IF EXISTS t_nul; @@ -41,7 +41,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); SELECT '====map===='; SET allow_experimental_map_type = 1; @@ -60,7 +60,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND current_database = currentDatabase(); DROP TABLE t_arr; DROP TABLE t_nul; diff --git a/tests/queries/0_stateless/01531_query_log_query_comment.sql b/tests/queries/0_stateless/01531_query_log_query_comment.sql index 2e1faf1b9e4..bed3a0fd95e 100644 --- a/tests/queries/0_stateless/01531_query_log_query_comment.sql +++ b/tests/queries/0_stateless/01531_query_log_query_comment.sql @@ -5,7 +5,7 @@ set enable_global_with_statement=1; select /* test=01531, enable_global_with_statement=0 */ 2; system flush logs; select count() from system.query_log -where event_time >= now() - interval 5 minute +where event_date >= yesterday() and query like '%select /* test=01531, enable_global_with_statement=0 */ 2%' and current_database = currentDatabase() ; @@ -14,7 +14,7 @@ set enable_global_with_statement=1; select /* test=01531 enable_global_with_statement=1 */ 2; system flush logs; select count() from system.query_log -where event_time >= now() - interval 5 minute +where event_date >= yesterday() and query like '%select /* test=01531 enable_global_with_statement=1 */ 2%' and current_database = currentDatabase() ; diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 38c80617334..1c6722268ac 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -36,7 +36,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND event_date >= yesterday() AND current_database = currentDatabase(); SYSTEM DROP MARK CACHE; SELECT col3.n2.s FROM nested FORMAT Null; @@ -46,7 +46,7 @@ SYSTEM FLUSH LOGS; SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] FROM system.query_log WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%')) - AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + AND event_date >= yesterday() AND current_database = currentDatabase(); DROP TABLE nested; diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql index f0f681288cf..e551942aeb5 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql @@ -15,16 +15,14 @@ where query like '%01546_log_queries_min_query_duration_ms-fast%' and query not like '%system.query_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); select count() from system.query_thread_log where query like '%01546_log_queries_min_query_duration_ms-fast%' and query not like '%system.query_thread_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- -- slow -- query logged @@ -40,8 +38,7 @@ where query like '%01546_log_queries_min_query_duration_ms-slow%' and query not like '%system.query_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- There at least two threads involved in a simple query -- (one thread just waits another, sigh) select count() == 2 @@ -50,5 +47,4 @@ where query like '%01546_log_queries_min_query_duration_ms-slow%' and query not like '%system.query_thread_log%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); diff --git a/tests/queries/0_stateless/01547_query_log_current_database.sql b/tests/queries/0_stateless/01547_query_log_current_database.sql index c0ad22163ba..755df2fab41 100644 --- a/tests/queries/0_stateless/01547_query_log_current_database.sql +++ b/tests/queries/0_stateless/01547_query_log_current_database.sql @@ -23,8 +23,7 @@ from system.query_log where query like '%01547_query_log_current_database%' and current_database = currentDatabase() - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- at least two threads for processing -- (but one just waits for another, sigh) diff --git a/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql b/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql index e80e84646be..028b3400d01 100644 --- a/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql +++ b/tests/queries/0_stateless/01548_query_log_query_execution_ms.sql @@ -13,8 +13,7 @@ where query like '%01548_query_log_query_execution_ms%' and current_database = currentDatabase() and query_duration_ms between 100 and 800 - and event_date = today() - and event_time >= now() - interval 1 minute; + and event_date >= yesterday(); -- at least two threads for processing -- (but one just waits for another, sigh) diff --git a/tests/queries/0_stateless/01600_benchmark_query.sh b/tests/queries/0_stateless/01600_benchmark_query.sh index a563c87a10f..1cf9cb23c3c 100755 --- a/tests/queries/0_stateless/01600_benchmark_query.sh +++ b/tests/queries/0_stateless/01600_benchmark_query.sh @@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err +LOG="$CLICKHOUSE_TMP/err-$CLICKHOUSE_DATABASE" +$CLICKHOUSE_BENCHMARK --iterations 10 --query "SELECT 1" 1>/dev/null 2>"$LOG" -cat "$CLICKHOUSE_TMP"/err | grep Exception -cat "$CLICKHOUSE_TMP"/err | grep Loaded +cat "$LOG" | grep Exception +cat "$LOG" | grep Loaded -rm "$CLICKHOUSE_TMP"/err +rm "$LOG" diff --git a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql index a1b419527d4..1f709f3c0c0 100644 --- a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql +++ b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql @@ -15,6 +15,8 @@ SYSTEM FLUSH LOGS; WITH ( SELECT (event_time, event_time_microseconds) FROM system.part_log + WHERE "table" = 'table_with_single_pk' + AND "database" = currentDatabase() ORDER BY event_time DESC LIMIT 1 ) AS time diff --git a/tests/queries/0_stateless/01702_system_query_log.sql b/tests/queries/0_stateless/01702_system_query_log.sql index f31d8de3577..5c3de9cf912 100644 --- a/tests/queries/0_stateless/01702_system_query_log.sql +++ b/tests/queries/0_stateless/01702_system_query_log.sql @@ -136,7 +136,7 @@ SELECT 'ACTUAL LOG CONTENT:'; -- Try to filter out all possible previous junk events by excluding old log entries, SELECT query_kind, query FROM system.query_log WHERE - log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_time >= now() - 10 + log_comment LIKE '%system.query_log%' AND type == 'QueryStart' AND event_date >= yesterday() AND current_database == currentDatabase() ORDER BY event_time_microseconds; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 981cf69d676..f89926b1b9e 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -524,8 +524,8 @@ "00571_non_exist_database_when_create_materializ_view", "00575_illegal_column_exception_when_drop_depen_column", "00599_create_view_with_subquery", - "00604_show_create_database", "00600_replace_running_query", + "00604_show_create_database", "00612_http_max_query_size", "00619_union_highlite", "00620_optimize_on_nonleader_replica_zookeeper", @@ -576,6 +576,7 @@ "00933_test_fix_extra_seek_on_compressed_cache", "00933_ttl_replicated_zookeeper", "00933_ttl_with_default", + "00950_dict_get", "00955_test_final_mark", "00976_ttl_with_old_parts", "00980_merge_alter_settings", @@ -738,8 +739,8 @@ "01530_drop_database_atomic_sync", "01541_max_memory_usage_for_user_long", "01542_dictionary_load_exception_race", - "01560_optimize_on_insert_zookeeper", "01545_system_errors", // looks at the difference of values in system.errors + "01560_optimize_on_insert_zookeeper", "01575_disable_detach_table_of_dictionary", "01593_concurrent_alter_mutations_kill", "01593_concurrent_alter_mutations_kill_many_replicas", @@ -752,11 +753,23 @@ "01603_rename_overwrite_bug", "01646_system_restart_replicas_smoke", // system restart replicas is a global query "01656_test_query_log_factories_info", + "01658_read_file_to_stringcolumn", "01669_columns_declaration_serde", "01676_dictget_in_default_expression", + "01681_cache_dictionary_simple_key", + "01682_cache_dictionary_complex_key", + "01683_flat_dictionary", + "01684_ssd_cache_dictionary_simple_key", + "01685_ssd_cache_dictionary_complex_key", "01700_system_zookeeper_path_in", + "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. + "01702_system_query_log", // Runs many global system queries "01715_background_checker_blather_zookeeper", + "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01747_alter_partition_key_enum_zookeeper", + "01748_dictionary_table_dot", // creates database + "01760_polygon_dictionaries", + "01760_system_dictionaries", "01761_alter_decimal_zookeeper", "attach", "ddl_dictionaries", @@ -765,18 +778,6 @@ "live_view", "memory_leak", "memory_limit", - "polygon_dicts", // they use an explicitly specified database - "01658_read_file_to_stringcolumn", - "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. - "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. - "01748_dictionary_table_dot", // creates database - "00950_dict_get", - "01683_flat_dictionary", - "01681_cache_dictionary_simple_key", - "01682_cache_dictionary_complex_key", - "01684_ssd_cache_dictionary_simple_key", - "01685_ssd_cache_dictionary_complex_key", - "01760_system_dictionaries", - "01760_polygon_dictionaries" + "polygon_dicts" // they use an explicitly specified database ] } From 3f273ef983a345d9eeaaae4054040a59fa1713d9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Mar 2021 21:01:56 +0300 Subject: [PATCH 085/266] Updated hash dictionary nullable attribute implementation --- src/Dictionaries/HashedDictionary.cpp | 217 +++++++++--------- src/Dictionaries/HashedDictionary.h | 21 +- .../01778_hierarchical_dictionaries.reference | 8 +- .../01778_hierarchical_dictionaries.sql | 60 ++--- 4 files changed, 149 insertions(+), 157 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 40394370087..427e9efca8e 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -2,6 +2,8 @@ #include +#include + #include #include #include @@ -155,98 +157,91 @@ ColumnPtr HashedDictionary::getColumn( const DictionaryAttribute & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); DefaultValueProvider default_value_provider(dictionary_attribute.null_value, default_values_column); + /// TODO: Check extractor performance DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); const auto & requested_keys = extractor.getKeys(); - auto result_column = dictionary_attribute.type->createColumn(); + auto result_column = dictionary_attribute.nested_type->createColumn(); size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; const auto & attribute = attributes[attribute_index]; size_t requested_keys_size = requested_keys.size(); - if (unlikely(attribute.is_complex_type)) + auto type_call = [&](const auto & dictionary_attribute_type) { - const auto & attribute_container = std::get>(attribute.container); + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnType = std::conditional_t< + std::is_same_v, + ColumnString, + std::conditional_t, ColumnDecimal, ColumnVector>>; - Field row_value_to_insert; + const auto & attribute_container = std::get>(attribute.container); + ColumnType & result_column_typed = static_cast(*result_column); + + if constexpr (std::is_same_v) + { + result_column_typed.reserve(requested_keys_size); + + for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) + { + auto & requested_key = requested_keys[requested_key_index]; + auto it = attribute_container.find(requested_key); + + if (it != attribute_container.end()) + { + StringRef item = getValueFromCell(it); + result_column->insertData(item.data, item.size); + } + else + { + Field default_value = default_value_provider.getDefaultValue(requested_key_index); + String & default_value_string = default_value.get(); + result_column->insertData(default_value_string.data(), default_value_string.size()); + } + } + } + else + { + auto & result_data = result_column_typed.getData(); + result_data.resize_fill(requested_keys_size); + + for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) + { + auto & requested_key = requested_keys[requested_key_index]; + + auto it = attribute_container.find(requested_key); + + if (it != attribute_container.end()) + { + ValueType item = getValueFromCell(it); + result_data[requested_key_index] = item; + } + else + { + auto default_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); + result_data[requested_key_index] = default_value_to_insert.get>(); + } + } + } + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (attribute.is_nullable_set) + { + ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(requested_keys_size, false); + ColumnUInt8::Container& vec_null_map_to = col_null_map_to->getData(); for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) { - auto & requested_key = requested_keys[requested_key_index]; - auto it = attribute_container.find(requested_key); - - if (it != attribute_container.end()) - row_value_to_insert = getValueFromCell(it); - else - row_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); - - result_column->insert(row_value_to_insert); + auto key = requested_keys[requested_key_index]; + vec_null_map_to[requested_key_index] = (attribute.is_nullable_set->find(key) != nullptr); } - } - else - { - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; - using ColumnType = std::conditional_t< - std::is_same_v, - ColumnString, - std::conditional_t, ColumnDecimal, ColumnVector>>; - const auto & attribute_container = std::get>(attribute.container); - ColumnType & result_column_typed = static_cast(*result_column); - - if constexpr (std::is_same_v) - { - result_column_typed.reserve(requested_keys_size); - - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) - { - auto & requested_key = requested_keys[requested_key_index]; - auto it = attribute_container.find(requested_key); - - if (it != attribute_container.end()) - { - StringRef item = getValueFromCell(it); - result_column->insertData(item.data, item.size); - } - else - { - Field default_value = default_value_provider.getDefaultValue(requested_key_index); - String & default_value_string = default_value.get(); - result_column->insertData(default_value_string.data(), default_value_string.size()); - } - } - } - else - { - auto & result_data = result_column_typed.getData(); - result_data.resize_fill(requested_keys_size); - - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) - { - auto & requested_key = requested_keys[requested_key_index]; - - auto it = attribute_container.find(requested_key); - - if (it != attribute_container.end()) - { - ValueType item = getValueFromCell(it); - result_data[requested_key_index] = item; - } - else - { - auto default_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); - result_data[requested_key_index] = default_value_to_insert.get>(); - } - } - } - }; - - callOnDictionaryAttributeType(attribute.type, type_call); + result_column = ColumnNullable::create(std::move(result_column), std::move(col_null_map_to)); } query_count.fetch_add(requested_keys.size(), std::memory_order_relaxed); @@ -260,6 +255,7 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co if (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); + /// TODO: Check performance of extractor Arena complex_keys_arena; DictionaryKeysExtractor extractor(key_columns, complex_keys_arena); @@ -275,14 +271,18 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co return result; } - /// Contaiiner + const auto & attribute = attributes.front(); getAttributeContainer(0, [&](const auto & container) { for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) { const auto & requested_key = keys[requested_key_index]; + out[requested_key_index] = container.find(requested_key) != container.end(); + + if (unlikely(attribute.is_nullable_set) && !out[requested_key_index]) + out[requested_key_index] = attribute.is_nullable_set->find(requested_key) != nullptr; } }); @@ -416,26 +416,17 @@ void HashedDictionary::createAttributes() for (const auto & dictionary_attribute : dict_struct.attributes) { - bool is_complex_type = dictionary_attribute.is_nullable || dictionary_attribute.is_array; - auto type_call = [&, this](const auto & dictionary_attribute_type) { using Type = std::decay_t; using AttributeType = typename Type::AttributeType; using ValueType = DictionaryValueType; + auto is_nullable_set = dictionary_attribute.is_nullable ? std::make_optional() : std::optional{}; std::unique_ptr string_arena = std::is_same_v ? std::make_unique() : nullptr; - if (is_complex_type) - { - Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, ComplexAttributeCollectionType(), std::move(string_arena)}; - attributes.emplace_back(std::move(attribute)); - } - else - { - Attribute attribute{dictionary_attribute.underlying_type, is_complex_type, CollectionType(), std::move(string_arena)}; - attributes.emplace_back(std::move(attribute)); - } + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), CollectionType(), std::move(string_arena)}; + attributes.emplace_back(std::move(attribute)); }; callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); @@ -567,6 +558,9 @@ void HashedDictionary::blockToAttributes(const Bloc auto key = keys_extracted_from_block[key_index]; auto it = container.find(key); + if (attribute.is_nullable_set && (attribute.is_nullable_set->find(key) != nullptr)) + continue; + if (it != container.end()) continue; @@ -575,11 +569,13 @@ void HashedDictionary::blockToAttributes(const Bloc attribute_column.get(key_index, column_value_to_insert); - if constexpr (std::is_same_v) + if (attribute.is_nullable_set && column_value_to_insert.isNull()) { - container.insert({key, column_value_to_insert}); + attribute.is_nullable_set->insert(key); + continue; } - else if constexpr (std::is_same_v) + + if constexpr (std::is_same_v) { String & value_to_insert = column_value_to_insert.get(); size_t value_to_insert_size = value_to_insert.size(); @@ -696,6 +692,9 @@ BlockInputStreamPtr HashedDictionary::getBlockInput PaddedPODArray keys; if (!attributes.empty()) + { + const auto & attribute = attributes.front(); + getAttributeContainer(0, [&](auto & container) { keys.reserve(container.size()); @@ -705,7 +704,17 @@ BlockInputStreamPtr HashedDictionary::getBlockInput (void)(value); keys.emplace_back(key); } + + if (attribute.is_nullable_set) + { + const auto & is_nullable_set = *attribute.is_nullable_set; + keys.reserve(is_nullable_set.size()); + + for (auto & node : is_nullable_set) + keys.emplace_back(node.getKey()); + } }); + } if constexpr (dictionary_key_type == DictionaryKeyType::simple) return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); @@ -721,25 +730,17 @@ void HashedDictionary::getAttributeContainer(size_t auto & attribute = attributes[attribute_index]; - if (unlikely(attribute.is_complex_type)) + auto type_call = [&](const auto & dictionary_attribute_type) { - auto & attribute_container = std::get>(attribute.container); + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + auto & attribute_container = std::get>(attribute.container); std::forward(get_container_func)(attribute_container); - } - else - { - auto type_call = [&](const auto & dictionary_attribute_type) - { - using Type = std::decay_t; - using AttributeType = typename Type::AttributeType; - using ValueType = DictionaryValueType; + }; - auto & attribute_container = std::get>(attribute.container); - std::forward(get_container_func)(attribute_container); - }; - - callOnDictionaryAttributeType(attribute.type, type_call); - } + callOnDictionaryAttributeType(attribute.type, type_call); } template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 3e7207e19cf..b12b07af444 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -116,12 +115,6 @@ private: HashMap, HashMapWithSavedHash>>; - template - using ComplexAttributeCollectionTypeNonSparse = std::conditional_t< - dictionary_key_type == DictionaryKeyType::simple, - absl::flat_hash_map>, - absl::flat_hash_map>>; - #if !defined(ARCADIA_BUILD) template using SparseHashMap = google::sparse_hash_map>; @@ -139,16 +132,13 @@ private: template using CollectionType = std::conditional_t, CollectionTypeNonSparse>; - template - using ComplexAttributeCollectionType = std::conditional_t< - sparse, - CollectionTypeSparse, - ComplexAttributeCollectionTypeNonSparse>; + using NullableSet = HashSet>; struct Attribute final { AttributeUnderlyingType type; - bool is_complex_type; + std::optional is_nullable_set; + std::variant< CollectionType, CollectionType, @@ -164,10 +154,11 @@ private: CollectionType, CollectionType, CollectionType, - CollectionType, - ComplexAttributeCollectionType> + CollectionType> container; + std::unique_ptr string_arena; + }; void createAttributes(); diff --git a/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference b/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference index 48245356ced..5fe5f5f1db6 100644 --- a/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference +++ b/tests/queries/0_stateless/01778_hierarchical_dictionaries.reference @@ -51,21 +51,21 @@ Get is in hierarchy 0 Get children [1] -[2,3] +[3,2] [4] [] [] [] Get all descendants -[1,2,3,4] -[2,3,4] +[1,3,2,4] +[3,2,4] [4] [] [] [] Get descendants at first level [1] -[2,3] +[3,2] [4] [] [] diff --git a/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql b/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql index eaf609e50fa..f6e1a7c9375 100644 --- a/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql +++ b/tests/queries/0_stateless/01778_hierarchical_dictionaries.sql @@ -1,95 +1,95 @@ DROP DATABASE IF EXISTS 01778_db; CREATE DATABASE 01778_db; -CREATE TABLE 01778_db.simple_key_hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog; -INSERT INTO 01778_db.simple_key_hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2); +CREATE TABLE 01778_db.hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog; +INSERT INTO 01778_db.hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2); -CREATE DICTIONARY 01778_db.simple_key_hierarchy_flat_dictionary +CREATE DICTIONARY 01778_db.hierarchy_flat_dictionary ( id UInt64, parent_id UInt64 HIERARCHICAL ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db')) LAYOUT(FLAT()) LIFETIME(MIN 1 MAX 1000); SELECT 'Flat dictionary'; SELECT 'Get hierarchy'; -SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetHierarchy('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get is in hierarchy'; -SELECT dictIsIn('01778_db.simple_key_hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT dictIsIn('01778_db.hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6; SELECT 'Get children'; -SELECT dictGetChildren('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetChildren('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get all descendants'; -SELECT dictGetDescendants('01778_db.simple_key_hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get descendants at first level'; -SELECT dictGetDescendants('01778_db.simple_key_hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6; +SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6; -DROP DICTIONARY 01778_db.simple_key_hierarchy_flat_dictionary; +DROP DICTIONARY 01778_db.hierarchy_flat_dictionary; -CREATE DICTIONARY 01778_db.simple_key_hierarchy_hashed_dictionary +CREATE DICTIONARY 01778_db.hierarchy_hashed_dictionary ( id UInt64, parent_id UInt64 HIERARCHICAL ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) -LAYOUT(FLAT()) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db')) +LAYOUT(HASHED()) LIFETIME(MIN 1 MAX 1000); SELECT 'Hashed dictionary'; SELECT 'Get hierarchy'; -SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetHierarchy('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get is in hierarchy'; -SELECT dictIsIn('01778_db.simple_key_hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT dictIsIn('01778_db.hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6; SELECT 'Get children'; -SELECT dictGetChildren('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetChildren('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get all descendants'; -SELECT dictGetDescendants('01778_db.simple_key_hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get descendants at first level'; -SELECT dictGetDescendants('01778_db.simple_key_hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6; +SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6; -DROP DICTIONARY 01778_db.simple_key_hierarchy_hashed_dictionary; +DROP DICTIONARY 01778_db.hierarchy_hashed_dictionary; -CREATE DICTIONARY 01778_db.simple_key_hierarchy_cache_dictionary +CREATE DICTIONARY 01778_db.hierarchy_cache_dictionary ( id UInt64, parent_id UInt64 HIERARCHICAL ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db')) LAYOUT(CACHE(SIZE_IN_CELLS 10)) LIFETIME(MIN 1 MAX 1000); SELECT 'Cache dictionary'; SELECT 'Get hierarchy'; -SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetHierarchy('01778_db.hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get is in hierarchy'; -SELECT dictIsIn('01778_db.simple_key_hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT dictIsIn('01778_db.hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6; -DROP DICTIONARY 01778_db.simple_key_hierarchy_cache_dictionary; +DROP DICTIONARY 01778_db.hierarchy_cache_dictionary; -CREATE DICTIONARY 01778_db.simple_key_hierarchy_direct_dictionary +CREATE DICTIONARY 01778_db.hierarchy_direct_dictionary ( id UInt64, parent_id UInt64 HIERARCHICAL ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_source_table' DB '01778_db')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db')) LAYOUT(DIRECT()); SELECT 'Direct dictionary'; SELECT 'Get hierarchy'; -SELECT dictGetHierarchy('01778_db.simple_key_hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6; +SELECT dictGetHierarchy('01778_db.hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6; SELECT 'Get is in hierarchy'; -SELECT dictIsIn('01778_db.simple_key_hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6; +SELECT dictIsIn('01778_db.hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6; -DROP DICTIONARY 01778_db.simple_key_hierarchy_direct_dictionary; +DROP DICTIONARY 01778_db.hierarchy_direct_dictionary; -DROP TABLE 01778_db.simple_key_hierarchy_source_table; +DROP TABLE 01778_db.hierarchy_source_table; DROP DATABASE 01778_db; From 75e1d657dcd68025ea7ea5552a26dbaa2739bf61 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Mar 2021 22:30:30 +0300 Subject: [PATCH 086/266] fixes --- .../01231_log_queries_min_type.sql | 15 +++++++++----- ...ference => 01304_direct_io_long.reference} | 0 ...4_direct_io.sh => 01304_direct_io_long.sh} | 0 .../01533_multiple_nested.reference | 20 +++++++++---------- .../0_stateless/01533_multiple_nested.sql | 2 +- ...og_queries_min_query_duration_ms.reference | 2 +- ...1546_log_queries_min_query_duration_ms.sql | 14 +++++-------- 7 files changed, 27 insertions(+), 26 deletions(-) rename tests/queries/0_stateless/{01304_direct_io.reference => 01304_direct_io_long.reference} (100%) rename tests/queries/0_stateless/{01304_direct_io.sh => 01304_direct_io_long.sh} (100%) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 382b7f0bf7e..66a3a6b967c 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -2,25 +2,30 @@ set log_queries=1; select '01231_log_queries_min_type/QUERY_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date >= yesterday(); +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/QUERY_START%' + and event_date >= yesterday(); set log_queries_min_type='EXCEPTION_BEFORE_START'; select '01231_log_queries_min_type/EXCEPTION_BEFORE_START'; system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date >= yesterday(); +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/EXCEPTION_BEFORE_START%' + and event_date >= yesterday(); set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; -select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; +select count() from system.query_log where current_database = currentDatabase() + and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' + and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; select count() from system.query_log where current_database = currentDatabase() and - query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and - query not like '%system.query_log%' and + query like 'select \'01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing' and has(Settings.Names, 'max_rows_to_read'); diff --git a/tests/queries/0_stateless/01304_direct_io.reference b/tests/queries/0_stateless/01304_direct_io_long.reference similarity index 100% rename from tests/queries/0_stateless/01304_direct_io.reference rename to tests/queries/0_stateless/01304_direct_io_long.reference diff --git a/tests/queries/0_stateless/01304_direct_io.sh b/tests/queries/0_stateless/01304_direct_io_long.sh similarity index 100% rename from tests/queries/0_stateless/01304_direct_io.sh rename to tests/queries/0_stateless/01304_direct_io_long.sh diff --git a/tests/queries/0_stateless/01533_multiple_nested.reference b/tests/queries/0_stateless/01533_multiple_nested.reference index ba37ce1c32c..138a7cfd7f2 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.reference +++ b/tests/queries/0_stateless/01533_multiple_nested.reference @@ -13,16 +13,16 @@ col3 read files 4 6 -0 899984 7199412 -1 899987 7199877 -2 899990 7200255 -3 899993 7199883 -4 899996 7199798 -5 899999 7200306 -6 900002 7200064 -7 900005 7199429 -8 900008 7200067 -9 899992 7199993 +0 89982 719752 +1 89988 720017 +2 89994 720152 +3 90000 720157 +4 90006 720100 +5 90012 720168 +6 90018 720106 +7 90005 719891 +8 89992 719854 +9 89979 719706 0 [] 0 [0] 1 [0,2] diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 1c6722268ac..0ddb0cfbfb4 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -59,7 +59,7 @@ ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; -INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(1000000); +INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(100000); SELECT id % 10, sum(length(col1)), sumArray(arrayMap(x -> length(x), col1.n.b)) FROM nested GROUP BY id % 10; SELECT arraySum(col1.a), arrayMap(x -> x * x * 2, col1.a) FROM nested ORDER BY id LIMIT 5; diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference index 0463db26710..8947c3c2bc3 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.reference @@ -1,4 +1,4 @@ 0 0 1 -1 +OK diff --git a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql index e551942aeb5..20854da0e8a 100644 --- a/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql +++ b/tests/queries/0_stateless/01546_log_queries_min_query_duration_ms.sql @@ -12,15 +12,13 @@ system flush logs; select count() from system.query_log where - query like '%01546_log_queries_min_query_duration_ms-fast%' - and query not like '%system.query_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-fast%' and current_database = currentDatabase() and event_date >= yesterday(); select count() from system.query_thread_log where - query like '%01546_log_queries_min_query_duration_ms-fast%' - and query not like '%system.query_thread_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-fast%' and current_database = currentDatabase() and event_date >= yesterday(); @@ -35,16 +33,14 @@ system flush logs; select count() from system.query_log where - query like '%01546_log_queries_min_query_duration_ms-slow%' - and query not like '%system.query_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-slow%' and current_database = currentDatabase() and event_date >= yesterday(); -- There at least two threads involved in a simple query -- (one thread just waits another, sigh) -select count() == 2 +select if(count() == 2, 'OK', 'Fail: ' || toString(count())) from system.query_thread_log where - query like '%01546_log_queries_min_query_duration_ms-slow%' - and query not like '%system.query_thread_log%' + query like 'select \'01546_log_queries_min_query_duration_ms-slow%' and current_database = currentDatabase() and event_date >= yesterday(); From 0faab7ae35e65cb5f49aea0488d7e55cbc52cf6a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Mar 2021 00:49:16 +0300 Subject: [PATCH 087/266] Fixed PODArray insertFromItself --- src/Common/PODArray.h | 8 +++++--- src/Dictionaries/HierarchyDictionariesUtils.h | 1 - tests/queries/skip_list.json | 3 ++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index 3d4050fbffa..b1042332cfa 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -538,17 +538,19 @@ public: /// Convert iterators to indexes because reserve can invalidate iterators size_t start_index = from_begin - begin(); size_t end_index = from_end - begin(); + size_t copy_size = end_index - start_index; assert(start_index <= end_index); - size_t required_capacity = this->size() + (end_index - start_index); + size_t required_capacity = this->size() + copy_size; if (required_capacity > this->capacity()) this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); - size_t bytes_to_copy = this->byte_size(end_index - start_index); + size_t bytes_to_copy = this->byte_size(copy_size); if (bytes_to_copy) { - memcpy(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); + auto begin = this->c_start + this->byte_size(start_index); + memcpy(this->c_end, reinterpret_cast(&*begin), bytes_to_copy); this->c_end += bytes_to_copy; } } diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index f8a1252f1b2..97200238661 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -112,7 +112,6 @@ namespace detail size_t start_index = previous_offset_size + array_element_offset; size_t end_index = offsets[offset]; - current_hierarchy_depth += end_index - start_index; elements.insertFromItself(elements.begin() + start_index, elements.begin() + end_index); break; } diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 38ad9dbe9a4..3ee00637513 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -778,6 +778,7 @@ "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", "01760_system_dictionaries", - "01760_polygon_dictionaries" + "01760_polygon_dictionaries", + "01778_hierarchical_dictionaries" ] } From ca6146b0400d6913e64537a4b8925149163c7df5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Mar 2021 13:42:46 +0300 Subject: [PATCH 088/266] Fixed build --- src/Dictionaries/HashedDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 427e9efca8e..6bd1d58ca99 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -347,7 +347,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; - auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; const UInt64 null_value = dictionary_attribute.null_value.get(); @@ -381,7 +381,7 @@ template ColumnPtr HashedDictionary::getDescendants( ColumnPtr key_column [[maybe_unused]], const DataTypePtr &, - size_t level) const + size_t level [[maybe_unused]]) const { if constexpr (dictionary_key_type == DictionaryKeyType::simple) { From 4f298d54b9823c9bd130d1b9a6207a8a586a3cc1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Mar 2021 17:00:00 +0300 Subject: [PATCH 089/266] Added perf tests --- src/Dictionaries/HierarchyDictionariesUtils.h | 2 +- tests/performance/direct_dictionary.xml | 92 +++++++++++-------- tests/performance/flat_dictionary.xml | 46 ++++++++++ tests/performance/hashed_dictionary.xml | 92 +++++++++++++++++++ 4 files changed, 191 insertions(+), 41 deletions(-) create mode 100644 tests/performance/flat_dictionary.xml create mode 100644 tests/performance/hashed_dictionary.xml diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 97200238661..8b2fe6ef08e 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -227,7 +227,7 @@ namespace detail PaddedPODArray descendants; descendants.reserve(keys_size); - PaddedPODArray descendants_offsets; + PaddedPODArray descendants_offsets; descendants_offsets.reserve(keys_size); struct Range diff --git a/tests/performance/direct_dictionary.xml b/tests/performance/direct_dictionary.xml index 68b52d917dd..2c12ba95534 100644 --- a/tests/performance/direct_dictionary.xml +++ b/tests/performance/direct_dictionary.xml @@ -1,38 +1,17 @@ - CREATE TABLE simple_direct_dictionary_test_table + CREATE TABLE simple_key_source_table ( id UInt64, value_int UInt64, value_string String, value_decimal Decimal64(8), value_string_nullable Nullable(String) - ) ENGINE = TinyLog; + ) ENGINE = Memory; - INSERT INTO simple_direct_dictionary_test_table - SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) - FROM system.numbers - LIMIT 100000; - - - - CREATE DICTIONARY simple_direct_dictionary - ( - id UInt64, - value_int UInt64, - value_string String, - value_decimal Decimal64(8), - value_string_nullable Nullable(String) - ) - PRIMARY KEY id - SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_direct_dictionary_test_table')) - LAYOUT(DIRECT()) - - - - CREATE TABLE complex_direct_dictionary_test_table + CREATE TABLE complex_key_source_table ( id UInt64, id_key String, @@ -44,14 +23,21 @@ - INSERT INTO complex_direct_dictionary_test_table - SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) - FROM system.numbers - LIMIT 100000; + CREATE DICTIONARY simple_key_direct_dictionary + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) + LAYOUT(DIRECT()) - CREATE DICTIONARY complex_direct_dictionary + CREATE DICTIONARY complex_key_direct_dictionary ( id UInt64, id_key String, @@ -61,20 +47,46 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id, id_key - SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_direct_dictionary_test_table')) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_source_table')) LAYOUT(COMPLEX_KEY_DIRECT()) - SELECT dictGet('default.simple_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.simple_direct_dictionary', number) FROM system.numbers LIMIT 150000; + + INSERT INTO simple_key_source_table + SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 100000; + - SELECT dictGet('default.complex_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.complex_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + + INSERT INTO complex_key_source_table + SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 100000; + + + SELECT dictGet('default.simple_key_source_table', 'value_int', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_source_table', 'value_string', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_source_table', 'value_decimal', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_source_table', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.simple_key_source_table', number) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.simple_key_source_table', 'value_int', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_source_table', 'value_string', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_source_table', 'value_decimal', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_source_table', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.simple_key_source_table', number) FROM system.numbers LIMIT 150000; + + SELECT dictGet('default.complex_key_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.complex_key_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; diff --git a/tests/performance/flat_dictionary.xml b/tests/performance/flat_dictionary.xml new file mode 100644 index 00000000000..fd2c6f953af --- /dev/null +++ b/tests/performance/flat_dictionary.xml @@ -0,0 +1,46 @@ + + + CREATE TABLE simple_key_source_table + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE DICTIONARY simple_key_flat_dictionary + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) + LAYOUT(FLAT()) + + + + INSERT INTO simple_key_source_table + SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 100000; + + + SELECT dictGet('default.simple_key_flat_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.simple_key_flat_dictionary', number) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.simple_key_flat_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_flat_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.simple_key_flat_dictionary', number) FROM system.numbers LIMIT 150000; + + diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml new file mode 100644 index 00000000000..113161d143b --- /dev/null +++ b/tests/performance/hashed_dictionary.xml @@ -0,0 +1,92 @@ + + + CREATE TABLE simple_key_source_table + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE TABLE complex_key_source_table + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = MEMORY; + + + + CREATE DICTIONARY simple_key_hashed_dictionary + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) + LAYOUT(DIRECT()) + + + + CREATE DICTIONARY complex_key_hashed_dictionary + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id, id_key + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_source_table')) + LAYOUT(COMPLEX_KEY_DIRECT()) + + + + INSERT INTO simple_key_source_table + SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 100000; + + + + INSERT INTO complex_key_source_table + SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 100000; + z + + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.simple_key_hashed_dictionary', number) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.simple_key_hashed_dictionary', number) FROM system.numbers LIMIT 150000; + + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + + From 2e55d47bab399d19eb6371111eecd5960658e8b3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 28 Mar 2021 13:35:26 +0300 Subject: [PATCH 090/266] Updated benchmarks --- tests/performance/direct_dictionary.xml | 38 ++++++++++++++----------- tests/performance/flat_dictionary.xml | 11 +++++-- tests/performance/hashed_dictionary.xml | 28 +++++++++++------- 3 files changed, 48 insertions(+), 29 deletions(-) diff --git a/tests/performance/direct_dictionary.xml b/tests/performance/direct_dictionary.xml index 2c12ba95534..80cbfc749ba 100644 --- a/tests/performance/direct_dictionary.xml +++ b/tests/performance/direct_dictionary.xml @@ -1,6 +1,6 @@ - CREATE TABLE simple_key_source_table + CREATE TABLE simple_key_direct_dictionary_source_table ( id UInt64, value_int UInt64, @@ -11,7 +11,7 @@ - CREATE TABLE complex_key_source_table + CREATE TABLE complex_key_direct_dictionary_source_table ( id UInt64, id_key String, @@ -32,7 +32,7 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id - SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_direct_dictionary_source_table')) LAYOUT(DIRECT()) @@ -47,35 +47,35 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id, id_key - SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_source_table')) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_direct_dictionary_source_table')) LAYOUT(COMPLEX_KEY_DIRECT()) - INSERT INTO simple_key_source_table + INSERT INTO simple_key_direct_dictionary_source_table SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers LIMIT 100000; - INSERT INTO complex_key_source_table + INSERT INTO complex_key_direct_dictionary_source_table SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers LIMIT 100000; - SELECT dictGet('default.simple_key_source_table', 'value_int', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_source_table', 'value_string', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_source_table', 'value_decimal', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_source_table', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.simple_key_source_table', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; + SELECT dictHas('default.simple_key_direct_dictionary', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_source_table', 'value_int', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_source_table', 'value_string', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_source_table', 'value_decimal', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_source_table', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.simple_key_source_table', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; + SELECT dictGet('default.simple_key_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; + SELECT dictHas('default.simple_key_direct_dictionary', number) FROM system.numbers LIMIT 150000; SELECT dictGet('default.complex_key_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 75000; SELECT dictGet('default.complex_key_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 75000; @@ -89,4 +89,10 @@ SELECT dictGet('default.complex_key_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + DROP TABLE IF EXISTS simple_key_direct_dictionary_source_table; + DROP TABLE IF EXISTS complex_key_direct_dictionary_source_table; + + DROP DICTIONARY IF EXISTS simple_key_direct_dictionary; + DROP DICTIONARY IF EXISTS complex_key_direct_dictionary; + diff --git a/tests/performance/flat_dictionary.xml b/tests/performance/flat_dictionary.xml index fd2c6f953af..02bbee17327 100644 --- a/tests/performance/flat_dictionary.xml +++ b/tests/performance/flat_dictionary.xml @@ -1,6 +1,6 @@ - CREATE TABLE simple_key_source_table + CREATE TABLE simple_key_flat_dictionary_source_table ( id UInt64, value_int UInt64, @@ -20,12 +20,13 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id - SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_flat_dictionary_source_table')) LAYOUT(FLAT()) + LIFETIME(MIN 0 MAX 1000) - INSERT INTO simple_key_source_table + INSERT INTO simple_key_flat_dictionary_source_table SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers LIMIT 100000; @@ -43,4 +44,8 @@ SELECT dictGet('default.simple_key_flat_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; SELECT dictHas('default.simple_key_flat_dictionary', number) FROM system.numbers LIMIT 150000; + DROP TABLE IF EXISTS simple_key_flat_dictionary_source_table + + DROP DICTIONARY IF EXISTS simple_key_flat_dictionary + diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index 113161d143b..b8ce1222408 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -1,6 +1,6 @@ - CREATE TABLE simple_key_source_table + CREATE TABLE simple_key_hashed_dictionary_source_table ( id UInt64, value_int UInt64, @@ -11,7 +11,7 @@ - CREATE TABLE complex_key_source_table + CREATE TABLE complex_key_hashed_dictionary_source_table ( id UInt64, id_key String, @@ -19,7 +19,7 @@ value_string String, value_decimal Decimal64(8), value_string_nullable Nullable(String) - ) ENGINE = MEMORY; + ) ENGINE = Memory; @@ -32,8 +32,9 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id - SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_source_table')) - LAYOUT(DIRECT()) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_hashed_dictionary_source_table')) + LAYOUT(HASHED()) + LIFETIME(MIN 0 MAX 1000); @@ -47,23 +48,24 @@ value_string_nullable Nullable(String) ) PRIMARY KEY id, id_key - SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_source_table')) - LAYOUT(COMPLEX_KEY_DIRECT()) + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_hashed_dictionary_source_table')) + LAYOUT(COMPLEX_KEY_HASHED()) + LIFETIME(MIN 0 MAX 1000); - INSERT INTO simple_key_source_table + INSERT INTO simple_key_hashed_dictionary_source_table SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers LIMIT 100000; - INSERT INTO complex_key_source_table + INSERT INTO complex_key_hashed_dictionary_source_table SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers LIMIT 100000; - z + SELECT dictGet('default.simple_key_hashed_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; @@ -89,4 +91,10 @@ SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + DROP TABLE IF EXISTS simple_key_hashed_dictionary_source_table; + DROP TABLE IF EXISTS complex_key_hashed_dictionary_source_table; + + DROP DICTIONARY IF EXISTS simple_key_hashed_dictionary; + DROP DICTIONARY IF EXISTS complex_key_hashed_dictionary; + From 704a4a552183739f4d84efe3e83b38d98f504e83 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 28 Mar 2021 20:43:21 +0300 Subject: [PATCH 091/266] Description of setting min_marks_to_honor_max_concurrent_queries --- .../settings.md | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 89fcbafe663..141a9a6b012 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -504,6 +504,11 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa The maximum number of simultaneously processed requests. +Possible values: + +- Positive integer. +- 0 — Disabled. + **Example** ``` xml @@ -530,6 +535,25 @@ Default value: `0` that means no limit. - [max_concurrent_queries](#max-concurrent-queries) +## min_marks_to_honor_max_concurrent_queries {#min-marks-to-honor-max-concurrent-queries} + +Minimal number of marks for applying the `max_concurrent_queries` setting. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +**Example** + +``` xml +10 +``` + +**See Also** + +- [max_concurrent_queries](#max-concurrent-queries) + ## max_connections {#max-connections} The maximum number of inbound connections. From 8d3f6be3704f7d34fe04a7adc7f60661e896310e Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 28 Mar 2021 21:26:51 +0300 Subject: [PATCH 092/266] minor fixes --- docs/en/sql-reference/functions/files.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 47b07d9ac9d..0e30d1bdd07 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -5,7 +5,7 @@ toc_title: Files # Functions for Working with Files {#functions-for-working-with-files} -## file(path) {#file} +## file {#file} Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from `user_files_path`. Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** @@ -26,7 +26,7 @@ Inserting data from files a.txt and b.txt into a table as different rows. Query: ``` sql -INSERT INTO table SELECT file('a.txt'), file('b.txt') +INSERT INTO table SELECT file('a.txt'), file('b.txt'); ``` **See Also** From 053b28ff7fc5eb62c3460b65823c04a7cec8ba78 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 14:39:27 +0300 Subject: [PATCH 093/266] fix --- tests/queries/0_stateless/01231_log_queries_min_type.sql | 1 + .../queries/0_stateless/01531_query_log_query_comment.sql | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index 66a3a6b967c..b6e27506b72 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -16,6 +16,7 @@ select count() from system.query_log where current_database = currentDatabase() set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } +set max_rows_to_read=0; system flush logs; select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' diff --git a/tests/queries/0_stateless/01531_query_log_query_comment.sql b/tests/queries/0_stateless/01531_query_log_query_comment.sql index bed3a0fd95e..19d6eac0e15 100644 --- a/tests/queries/0_stateless/01531_query_log_query_comment.sql +++ b/tests/queries/0_stateless/01531_query_log_query_comment.sql @@ -1,20 +1,20 @@ set log_queries=1; set log_queries_min_type='QUERY_FINISH'; -set enable_global_with_statement=1; +set enable_global_with_statement=0; select /* test=01531, enable_global_with_statement=0 */ 2; system flush logs; select count() from system.query_log where event_date >= yesterday() - and query like '%select /* test=01531, enable_global_with_statement=0 */ 2%' + and query like 'select /* test=01531, enable_global_with_statement=0 */ 2%' and current_database = currentDatabase() ; set enable_global_with_statement=1; -select /* test=01531 enable_global_with_statement=1 */ 2; +select /* test=01531, enable_global_with_statement=1 */ 2; system flush logs; select count() from system.query_log where event_date >= yesterday() - and query like '%select /* test=01531 enable_global_with_statement=1 */ 2%' + and query like 'select /* test=01531, enable_global_with_statement=1 */ 2%' and current_database = currentDatabase() ; From 7df765e4b4fdfa5799c3489a618f7d4f212f9643 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 14:47:57 +0300 Subject: [PATCH 094/266] save query log --- docker/test/stateless/run.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3119ae27c59..853814faae8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -74,7 +74,13 @@ timeout "$MAX_RUN_TIME" bash -c run_tests ||: ./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv -pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz ||: +clickhouse-client -q "sytem flush logs" ||: + +pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz & +clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz & +clickhouse-client -q "select * from system.query_thread_log format TSVWithNamesAndTypes" | pigz > /test_output/query-thread-log.tsv.gz & +wait ||: + mv /var/log/clickhouse-server/stderr.log /test_output/ ||: if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: From 1e4ddae457e3364f62c3b7ab52a8be5a63b14582 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 16:11:21 +0300 Subject: [PATCH 095/266] fix --- tests/queries/0_stateless/01231_log_queries_min_type.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index b6e27506b72..b3540f3354b 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -22,8 +22,10 @@ select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; +set max_rows_to_read='100K'; select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } system flush logs; +set max_rows_to_read=0; select count() from system.query_log where current_database = currentDatabase() and query like 'select \'01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and From 3071d45ade870975df33c8d99a06c7822b96343c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 29 Mar 2021 16:41:03 +0300 Subject: [PATCH 096/266] Fixed tests --- .../HierarchyDictionariesUtils.cpp | 44 ++++++------- tests/performance/direct_dictionary.xml | 64 +++++++++++++------ tests/performance/flat_dictionary.xml | 45 +++++++++---- tests/performance/hashed_dictionary.xml | 62 +++++++++++------- 4 files changed, 139 insertions(+), 76 deletions(-) diff --git a/src/Dictionaries/HierarchyDictionariesUtils.cpp b/src/Dictionaries/HierarchyDictionariesUtils.cpp index 99ff7aa63fa..fffe0d30e0e 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.cpp +++ b/src/Dictionaries/HierarchyDictionariesUtils.cpp @@ -18,11 +18,11 @@ namespace */ HashMap getChildToParentHierarchyMapImpl( const IDictionary * dictionary, - const DictionaryAttribute & dictionary_attribute, + const DictionaryAttribute & hierarchical_attribute, const PaddedPODArray & initial_keys_to_request, const DataTypePtr & key_type) { - UInt64 null_value = dictionary_attribute.null_value.get(); + UInt64 null_value = hierarchical_attribute.null_value.get(); ColumnPtr key_to_request_column = ColumnVector::create(); auto * key_to_request_column_typed = static_cast *>(key_to_request_column->assumeMutable().get()); @@ -42,12 +42,18 @@ namespace { child_to_parent_key.reserve(child_to_parent_key.size() + keys_to_request.size()); - auto parent_key_column - = dictionary->getColumn(dictionary_attribute.name, dictionary_attribute.type, {key_to_request_column}, {key_type}, key_not_in_storage_default_value_column); + auto parent_key_column = dictionary->getColumn( + hierarchical_attribute.name, + hierarchical_attribute.type, + {key_to_request_column}, + {key_type}, + key_not_in_storage_default_value_column); const auto * parent_key_column_typed = checkAndGetColumn>(*parent_key_column); if (!parent_key_column_typed) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Parent key column should be UInt64"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Parent key column should be UInt64. Actual ({})", + hierarchical_attribute.type->getName()); const auto & parent_keys = parent_key_column_typed->getData(); next_keys_to_request.clear(); @@ -86,26 +92,22 @@ ColumnPtr getKeysHierarchyDefaultImplementation(const IDictionary * dictionary, throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); const auto & dictionary_structure = dictionary->getStructure(); - const auto & dictionary_attribute = dictionary_structure.attributes[0]; + size_t hierarchical_attribute_index = *dictionary_structure.hierarchical_attribute_index; + const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index]; const PaddedPODArray & requested_keys = key_column_typed->getData(); - HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, hierarchical_attribute, requested_keys, key_type); auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; auto get_parent_key_func = [&](auto & key) { - std::optional result; - auto it = key_to_parent_key.find(key); - - if (it != nullptr) - result = it->getMapped(); - + std::optional result = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); return result; }; - UInt64 null_value = dictionary_attribute.null_value.get(); + UInt64 null_value = hierarchical_attribute.null_value.get(); auto dictionary_hierarchy_array = getKeysHierarchyArray(requested_keys, null_value, is_key_valid_func, get_parent_key_func); return dictionary_hierarchy_array; @@ -129,26 +131,22 @@ ColumnUInt8::Ptr getKeysIsInHierarchyDefaultImplementation( throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Key column should be UInt64"); const auto & dictionary_structure = dictionary->getStructure(); - const auto & dictionary_attribute = dictionary_structure.attributes[0]; + size_t hierarchical_attribute_index = *dictionary_structure.hierarchical_attribute_index; + const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index]; const PaddedPODArray & requested_keys = key_column_typed->getData(); - HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, dictionary_attribute, requested_keys, key_type); + HashMap key_to_parent_key = getChildToParentHierarchyMapImpl(dictionary, hierarchical_attribute, requested_keys, key_type); auto is_key_valid_func = [&](auto & key) { return key_to_parent_key.find(key) != nullptr; }; auto get_parent_key_func = [&](auto & key) { - std::optional result; - auto it = key_to_parent_key.find(key); - - if (it != nullptr) - result = it->getMapped(); - + std::optional result = (it != nullptr ? std::make_optional(it->getMapped()) : std::nullopt); return result; }; - UInt64 null_value = dictionary_attribute.null_value.get(); + UInt64 null_value = hierarchical_attribute.null_value.get(); const auto & in_keys = in_key_column_typed->getData(); auto result = getKeysIsInHierarchyColumn(requested_keys, in_keys, null_value, is_key_valid_func, get_parent_key_func); diff --git a/tests/performance/direct_dictionary.xml b/tests/performance/direct_dictionary.xml index 80cbfc749ba..97ecdfe3e95 100644 --- a/tests/performance/direct_dictionary.xml +++ b/tests/performance/direct_dictionary.xml @@ -65,29 +65,53 @@ LIMIT 100000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.simple_key_direct_dictionary', number) FROM system.numbers LIMIT 75000; + + + column_name + + 'value_int' + 'value_string' + 'value_decimal' + 'value_string_nullable' + + - SELECT dictGet('default.simple_key_direct_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_direct_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.simple_key_direct_dictionary', number) FROM system.numbers LIMIT 150000; + + elements_count + + 25000 + 50000 + 75000 + 100000 + + + - SELECT dictGet('default.complex_key_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.simple_key_direct_dictionary', {column_name}, number) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + SELECT dictHas('default.simple_key_direct_dictionary', number) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + - SELECT dictGet('default.complex_key_direct_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_direct_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + + SELECT dictGet('default.complex_key_direct_dictionary', {column_name}, (number, toString(number))) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + SELECT dictHas('default.complex_key_direct_dictionary', (number, toString(number))) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + DROP TABLE IF EXISTS simple_key_direct_dictionary_source_table; DROP TABLE IF EXISTS complex_key_direct_dictionary_source_table; diff --git a/tests/performance/flat_dictionary.xml b/tests/performance/flat_dictionary.xml index 02bbee17327..6eb4bc2be8f 100644 --- a/tests/performance/flat_dictionary.xml +++ b/tests/performance/flat_dictionary.xml @@ -29,20 +29,43 @@ INSERT INTO simple_key_flat_dictionary_source_table SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers - LIMIT 100000; + LIMIT 500000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.simple_key_flat_dictionary', number) FROM system.numbers LIMIT 75000; + + + column_name + + 'value_int' + 'value_string' + 'value_decimal' + 'value_string_nullable' + + - SELECT dictGet('default.simple_key_flat_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_flat_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.simple_key_flat_dictionary', number) FROM system.numbers LIMIT 150000; + + elements_count + + 250000 + 500000 + 750000 + 1000000 + + + + + + SELECT dictGet('default.simple_key_flat_dictionary', {column_name}, number) + FROM system.numbers + LIMIT {elements_count}; + + + + SELECT dictHas('default.simple_key_flat_dictionary', number) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + DROP TABLE IF EXISTS simple_key_flat_dictionary_source_table diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index b8ce1222408..150b0309ce4 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -57,39 +57,57 @@ INSERT INTO simple_key_hashed_dictionary_source_table SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers - LIMIT 100000; + LIMIT 5000000; INSERT INTO complex_key_hashed_dictionary_source_table SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) FROM system.numbers - LIMIT 100000; + LIMIT 5000000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_int', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.simple_key_hashed_dictionary', number) FROM system.numbers LIMIT 75000; + + + column_name + + 'value_int' + 'value_string' + 'value_decimal' + 'value_string_nullable' + + - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_int', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_decimal', number) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.simple_key_hashed_dictionary', 'value_string_nullable', number) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.simple_key_hashed_dictionary', number) FROM system.numbers LIMIT 150000; + + elements_count + + 2500000 + 5000000 + 7500000 + 10000000 + + + - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 75000; - SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) FROM system.numbers LIMIT 75000; + + SELECT dictGet('default.simple_key_hashed_dictionary', {column_name}, number) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + SELECT dictHas('default.simple_key_hashed_dictionary', number) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_int', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_decimal', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictGet('default.complex_key_hashed_dictionary', 'value_string_nullable', (number, toString(number))) FROM system.numbers LIMIT 150000; - SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) FROM system.numbers LIMIT 150000; + + SELECT dictGet('default.complex_key_hashed_dictionary', {column_name}, (number, toString(number))) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + DROP TABLE IF EXISTS simple_key_hashed_dictionary_source_table; DROP TABLE IF EXISTS complex_key_hashed_dictionary_source_table; From 0cb85b9ea89c5cc6e790ed7dd86d3f30dca01ed8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Mar 2021 17:26:48 +0300 Subject: [PATCH 097/266] Trying parallel functional tests one more time --- docker/test/stateless/run.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 853814faae8..80199b0fe92 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -53,10 +53,11 @@ function run_tests() if [ "$NUM_TRIES" -gt "1" ]; then ADDITIONAL_OPTIONS+=('--skip') ADDITIONAL_OPTIONS+=('00000_no_tests_to_skip') - ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('4') fi + ADDITIONAL_OPTIONS+=('--jobs') + ADDITIONAL_OPTIONS+=('8') + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') fi From 4f2627f3853b73551b4f97becc93ae25d337bb22 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 19:25:13 +0300 Subject: [PATCH 098/266] make it faster --- .../0_stateless/01686_event_time_microseconds_part_log.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql index 1f709f3c0c0..4a653379ef1 100644 --- a/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql +++ b/tests/queries/0_stateless/01686_event_time_microseconds_part_log.sql @@ -8,7 +8,7 @@ CREATE TABLE table_with_single_pk ENGINE = MergeTree ORDER BY key; -INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(10000000); +INSERT INTO table_with_single_pk SELECT number, toString(number % 10) FROM numbers(1000000); SYSTEM FLUSH LOGS; From fe2a364e098ba94f3c21fa9067d5faa17911efae Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 19:26:50 +0300 Subject: [PATCH 099/266] faster --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1604c08dcb6..5a8899e2658 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -448,7 +448,7 @@ def run_tests_array(all_tests_with_params): failures_chain += 1 status += MSG_FAIL status += print_test_time(total_time) - status += " - Long test not marked as 'long'" + status += " - Test runs too long (> 30s). Make it faster." else: passed_total += 1 failures_chain = 0 From d91736f36b33bd2e8a2b43a98204630be3cfbd91 Mon Sep 17 00:00:00 2001 From: George Date: Mon, 29 Mar 2021 20:46:44 +0300 Subject: [PATCH 100/266] updated postgresql.md --- docs/ru/engines/table-engines/integrations/postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md index ecf431830f8..1fc7a307d94 100644 --- a/docs/ru/engines/table-engines/integrations/postgresql.md +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -3,7 +3,7 @@ toc_priority: 8 toc_title: PostgreSQL --- -# PosgtreSQL {#postgresql} +# PostgreSQL {#postgresql} Движок PostgreSQL позволяет выполнять запросы `SELECT` над данными, хранящимися на удалённом PostgreSQL сервере. From ae741c7411a1f9faadc450ae26a62955d37be81f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 29 Mar 2021 20:49:28 +0300 Subject: [PATCH 101/266] Do not limit HTTP chunk size --- src/IO/HTTPChunkedReadBuffer.cpp | 4 ---- src/IO/HTTPChunkedReadBuffer.h | 3 +-- src/Server/HTTP/HTTPServerRequest.cpp | 3 +-- 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/IO/HTTPChunkedReadBuffer.cpp b/src/IO/HTTPChunkedReadBuffer.cpp index bd9bbba4c6c..374e04031d0 100644 --- a/src/IO/HTTPChunkedReadBuffer.cpp +++ b/src/IO/HTTPChunkedReadBuffer.cpp @@ -14,7 +14,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int UNEXPECTED_END_OF_FILE; extern const int CORRUPTED_DATA; - extern const int TOO_MANY_BYTES; } size_t HTTPChunkedReadBuffer::readChunkHeader() @@ -40,9 +39,6 @@ size_t HTTPChunkedReadBuffer::readChunkHeader() if (in->eof()) throw Exception("Unexpected end of file while reading chunk header of HTTP chunked data", ErrorCodes::UNEXPECTED_END_OF_FILE); - if (res > max_size) - throw Exception("Chunk size is too large", ErrorCodes::TOO_MANY_BYTES); - assertString("\n", *in); return res; } diff --git a/src/IO/HTTPChunkedReadBuffer.h b/src/IO/HTTPChunkedReadBuffer.h index 0ccebc69d08..378835cafc0 100644 --- a/src/IO/HTTPChunkedReadBuffer.h +++ b/src/IO/HTTPChunkedReadBuffer.h @@ -10,11 +10,10 @@ namespace DB class HTTPChunkedReadBuffer : public BufferWithOwnMemory { public: - HTTPChunkedReadBuffer(std::unique_ptr in_, size_t max_chunk_size) : in(std::move(in_)), max_size(max_chunk_size) {} + explicit HTTPChunkedReadBuffer(std::unique_ptr in_) : in(std::move(in_)) {} private: std::unique_ptr in; - const size_t max_size; size_t readChunkHeader(); void readChunkFooter(); diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index bdba6a51d91..2a765f36fd7 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -26,7 +26,6 @@ HTTPServerRequest::HTTPServerRequest(const Context & context, HTTPServerResponse auto receive_timeout = context.getSettingsRef().http_receive_timeout; auto send_timeout = context.getSettingsRef().http_send_timeout; - auto max_query_size = context.getSettingsRef().max_query_size; session.socket().setReceiveTimeout(receive_timeout); session.socket().setSendTimeout(send_timeout); @@ -37,7 +36,7 @@ HTTPServerRequest::HTTPServerRequest(const Context & context, HTTPServerResponse readRequest(*in); /// Try parse according to RFC7230 if (getChunkedTransferEncoding()) - stream = std::make_unique(std::move(in), max_query_size); + stream = std::make_unique(std::move(in)); else if (hasContentLength()) stream = std::make_unique(std::move(in), getContentLength(), false); else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) From 7f469b6c3172e5e88c9365f70f84d195dcfc3fb9 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 20:53:48 +0300 Subject: [PATCH 102/266] blns test zk path --- tests/queries/0_stateless/01666_blns.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01666_blns.sql b/tests/queries/0_stateless/01666_blns.sql index 787f0369244..5d7cc4881bb 100644 --- a/tests/queries/0_stateless/01666_blns.sql +++ b/tests/queries/0_stateless/01666_blns.sql @@ -554,9 +554,9 @@ SELECT count() FROM test; DROP TABLE IF EXISTS test_r1; DROP TABLE IF EXISTS test_r2; -CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test/01666_blns', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; INSERT INTO test_r1 SELECT * FROM test; -CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; +CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test/01666_blns', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G'; SYSTEM SYNC REPLICA test_r2; From fbbf4fbc63c14c14cc3c89b435572ee2bbe76cb4 Mon Sep 17 00:00:00 2001 From: George Date: Mon, 29 Mar 2021 20:54:28 +0300 Subject: [PATCH 103/266] updated column.md --- docs/ru/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index f51859b46f6..87fc1c78cd0 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -117,7 +117,7 @@ MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL] [AFTER name_after | F - TTL - Примеры изменения TTL столбца смотрите в разделе [TTL столбца](ttl.md#mergetree-column-ttl). + Примеры изменения TTL столбца смотрите в разделе [TTL столбца](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). Если указано `IF EXISTS`, запрос не возвращает ошибку, если столбца не существует. From a057a66b085f88d16388e47cfdb883e887113262 Mon Sep 17 00:00:00 2001 From: George Date: Mon, 29 Mar 2021 21:05:31 +0300 Subject: [PATCH 104/266] updated settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3421afee15d..b9b7d061697 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1876,7 +1876,7 @@ Enables or disables random shard insertion into a [Distributed](../../engines/ta By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. -Possible values: +**Possible values** - 0 — Insertion is rejected if there are multiple shards and no distributed key is given. - 1 — Insertion is done randomly among all available shards when no distributed key is given. From f43bb61a26b8c47692d0adbc05b98d7a4cd58791 Mon Sep 17 00:00:00 2001 From: George Date: Mon, 29 Mar 2021 21:11:12 +0300 Subject: [PATCH 105/266] updated settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b9b7d061697..3421afee15d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1876,7 +1876,7 @@ Enables or disables random shard insertion into a [Distributed](../../engines/ta By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. -**Possible values** +Possible values: - 0 — Insertion is rejected if there are multiple shards and no distributed key is given. - 1 — Insertion is done randomly among all available shards when no distributed key is given. From 112cd107aa7df09ac52450614bd137389a2320ae Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 21:14:06 +0300 Subject: [PATCH 106/266] print elapsed time per test group --- tests/clickhouse-test | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5a8899e2658..6800e8fa887 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -285,6 +285,7 @@ def run_tests_array(all_tests_with_params): failures_total = 0 failures = 0 failures_chain = 0 + start_time = datetime.now() client_options = get_additional_client_options(args) @@ -479,12 +480,14 @@ def run_tests_array(all_tests_with_params): failures_total = failures_total + failures if failures_total > 0: - print(colored("\nHaving {failures_total} errors! {passed_total} tests passed. {skipped_total} tests skipped.".format( - passed_total = passed_total, skipped_total = skipped_total, failures_total = failures_total), args, "red", attrs=["bold"])) + print(colored(f"\nHaving {failures_total} errors! {passed_total} tests passed." + f" {skipped_total} tests skipped. {(datetime.now() - start_time).total_seconds():.2f} s elapsed.", + args, "red", attrs=["bold"])) exit_code = 1 else: - print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format( - passed_total = passed_total, skipped_total = skipped_total), args, "green", attrs=["bold"])) + print(colored(f"\n{passed_total} tests passed. {skipped_total} tests skipped." + f" {(datetime.now() - start_time).total_seconds():.2f} s elapsed.", + args, "green", attrs=["bold"])) server_logs_level = "warning" From 72f8b42d63c82808c2baeb8cc058dbd9dd944e79 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 29 Mar 2021 21:19:13 +0300 Subject: [PATCH 107/266] more info --- tests/clickhouse-test | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6800e8fa887..2677adb1ff5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -287,6 +287,8 @@ def run_tests_array(all_tests_with_params): failures_chain = 0 start_time = datetime.now() + is_concurrent = multiprocessing.current_process().name != "MainProcess" + client_options = get_additional_client_options(args) def print_test_time(test_time): @@ -296,7 +298,7 @@ def run_tests_array(all_tests_with_params): return '' if all_tests: - print("\nRunning {} {} tests.".format(len(all_tests), suite) + "\n") + print(f"\nRunning {len(all_tests)} {suite} tests ({multiprocessing.current_process().name}).\n") for case in all_tests: if SERVER_DIED: @@ -311,7 +313,6 @@ def run_tests_array(all_tests_with_params): try: status = '' - is_concurrent = multiprocessing.current_process().name != "MainProcess" if not is_concurrent: sys.stdout.flush() sys.stdout.write("{0:72}".format(name + ": ")) @@ -481,12 +482,14 @@ def run_tests_array(all_tests_with_params): if failures_total > 0: print(colored(f"\nHaving {failures_total} errors! {passed_total} tests passed." - f" {skipped_total} tests skipped. {(datetime.now() - start_time).total_seconds():.2f} s elapsed.", + f" {skipped_total} tests skipped. {(datetime.now() - start_time).total_seconds():.2f} s elapsed" + f' ({multiprocessing.current_process().name}).', args, "red", attrs=["bold"])) exit_code = 1 else: print(colored(f"\n{passed_total} tests passed. {skipped_total} tests skipped." - f" {(datetime.now() - start_time).total_seconds():.2f} s elapsed.", + f" {(datetime.now() - start_time).total_seconds():.2f} s elapsed" + f' ({multiprocessing.current_process().name}).', args, "green", attrs=["bold"])) From 83b8f4e2ea8832782e8e871eb0049bccf3eb569d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 29 Mar 2021 22:27:06 +0300 Subject: [PATCH 108/266] Translate to Russian --- docs/ru/sql-reference/functions/files.md | 34 ++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 docs/ru/sql-reference/functions/files.md diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md new file mode 100644 index 00000000000..7252a3a18e2 --- /dev/null +++ b/docs/ru/sql-reference/functions/files.md @@ -0,0 +1,34 @@ +--- +toc_priority: 43 +toc_title: Files +--- + +# Функции для работы с файлами {#funktsii-dlia-raboty-s-failami} + +## file {#file} + +Читает файл как строку. Файл может содержать подзапросы, условие, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. + +**Синтаксис** + +``` sql +file(path) +``` + +**Аргументы** + +- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. + +**Примеры** + +Вставка данных из файлов a.txt и b.txt в таблицу в виде отдельных строк: + +``` sql +INSERT INTO table SELECT file('a.txt'), file('b.txt'); +``` + +**Смотрите также** + +- [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) +- [file](../table-functions/file.md) + From 8a65e8b06e383926c8e9bfe92f97b9b550b4177e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 29 Mar 2021 23:00:40 +0300 Subject: [PATCH 109/266] Fix arcadia build --- src/Dictionaries/HashedDictionary.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index b12b07af444..c7cb59c2e3f 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -119,8 +119,8 @@ private: template using SparseHashMap = google::sparse_hash_map>; #else - template - using SparseHashMap = google::sparsehash::sparse_hash_map>; + template + using SparseHashMap = google::sparsehash::sparse_hash_map>; #endif template From 9925110713c47611bccc8185a830b7f5c7ca0341 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 29 Mar 2021 23:04:50 +0300 Subject: [PATCH 110/266] rename other occurrances --- .../config/config.d/zzz-perf-comparison-tweaks-config.xml | 2 +- docker/test/stateful/run.sh | 4 ++-- docker/test/stateless/run.sh | 4 ++-- tests/config/config.d/database_replicated.xml | 4 ++-- tests/config/install.sh | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index ee2006201b0..31f5b739c6d 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -3,7 +3,7 @@ - + :: diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3682686bc99..9e210dc92a2 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -22,14 +22,14 @@ function start() --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ - --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 + --keeper_server.tcp_port 19181 --keeper_server.server_id 2 sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ - --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 + --keeper_server.tcp_port 29181 --keeper_server.server_id 3 fi counter=0 diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index fda9e61dd29..539b8d52cdd 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,7 +45,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ - --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ + --keeper_server.tcp_port 19181 --keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ @@ -53,7 +53,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ - --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ + --keeper_server.tcp_port 29181 --keeper_server.server_id 3 \ --macros.shard s2 # It doesn't work :( MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 3bbae6031c9..ed5845bad48 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -14,7 +14,7 @@ - + 9181 1 @@ -50,7 +50,7 @@ 1
- + diff --git a/tests/config/install.sh b/tests/config/install.sh index d5396179de9..9c4f8caca07 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -67,7 +67,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ rm /etc/clickhouse-server/config.d/zookeeper.xml - rm /etc/clickhouse-server/config.d/test_keeper_port.xml + rm /etc/clickhouse-server/config.d/keeper_port.xml # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 # And we have to copy configs... From 58987a973d9c020d31f4ed61bab3e8ebc5437faa Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 30 Mar 2021 01:41:07 +0300 Subject: [PATCH 111/266] flush --- tests/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2677adb1ff5..57f91b45b35 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -492,6 +492,8 @@ def run_tests_array(all_tests_with_params): f' ({multiprocessing.current_process().name}).', args, "green", attrs=["bold"])) + sys.stdout.flush() + server_logs_level = "warning" From eb62b1cfdbbf6f5d0ff9a9f8e0a8a8008a53338e Mon Sep 17 00:00:00 2001 From: George Date: Tue, 30 Mar 2021 02:01:04 +0300 Subject: [PATCH 112/266] translation draft --- docs/ru/operations/settings/settings.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 9d3ea4a809a..7509cb3ddb8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1760,6 +1760,21 @@ ClickHouse генерирует исключение - [Движок Distributed](../../engines/table-engines/special/distributed.md#distributed) - [Управление распределёнными таблицами](../../sql-reference/statements/system.md#query-language-system-distributed) +## insert_distributed_one_random_shard {#insert_distributed_one_random_shard} + +Включает или отключает режим вставки данных в [Distributed](../../engines/table-engines/special/distributed.md#distributed)) таблицу в случайный шард при отсутствии ключ шардирования. + +По умолчанию при вставке данных в `Distributed` таблицу с несколькими шардами и при отсутствии ключа шардирования сервер ClickHouse будет отклонять любой запрос на вставку данных. Когда `insert_distributed_one_random_shard = 1`, добавления принимаются, а данные записываются в случайный шард. + + +Возможные значения: + + +- 0 — если у таблицы несколько шардов, но ключ шардирования отсутствует, вставка данных отклоняется. +- 1 — если ключ шардирования отсутствует, то вставка данных осуществляется в случайный шард среди всех доступных шардов. + +Значение по умолчанию: `0`. + ## insert_shard_id {#insert_shard_id} Если не `0`, указывает, в какой шард [Distributed](../../engines/table-engines/special/distributed.md#distributed) таблицы данные будут вставлены синхронно. From 6d2f4dd82eaaf3615a7a6f40f6c7b1487c85f0bd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 30 Mar 2021 14:51:19 +0800 Subject: [PATCH 113/266] MySQL is started only once with MaterializeMySQL integration test --- .../test_materialize_mysql_database/test.py | 65 +++++++++++++------ 1 file changed, 44 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index ced9a978d02..730305a6f16 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -42,7 +42,7 @@ class MySQLNodeInstance: if not os.path.exists(self.instances_dir): os.mkdir(self.instances_dir) self.docker_logs_path = p.join(self.instances_dir, 'docker_mysql.log') - + self.start_up = False def alloc_connection(self): if self.mysql_connection is None: @@ -78,12 +78,16 @@ class MySQLNodeInstance: return cursor.fetchall() def start_and_wait(self): + if self.start_up: + return + run_and_check(['docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, - 'up', '--no-recreate', '-d', - ]) + '-p', cluster.project_name, + '-f', self.docker_compose, + 'up', '--no-recreate', '-d', + ]) self.wait_mysql_to_start(120) + self.start_up = True def close(self): if self.mysql_connection is not None: @@ -99,6 +103,8 @@ class MySQLNodeInstance: except Exception as e: print("Unable to get logs from docker mysql.") + self.start_up = False + def wait_mysql_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: @@ -113,32 +119,32 @@ class MySQLNodeInstance: run_and_check(['docker-compose', 'ps', '--services', 'all']) raise Exception("Cannot wait MySQL container") + +mysql_5_7_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') +mysql_5_7_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, mysql_5_7_docker_compose) + +mysql_8_0_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') +mysql_8_0_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, mysql_8_0_docker_compose) + + @pytest.fixture(scope="module") def started_mysql_5_7(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose) - try: - mysql_node.start_and_wait() - yield mysql_node + mysql_5_7_node.start_and_wait() + yield mysql_5_7_node finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) + mysql_5_7_node.close() + run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_5_7_docker_compose, 'down', '--volumes', '--remove-orphans']) @pytest.fixture(scope="module") def started_mysql_8_0(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose) - try: - mysql_node.start_and_wait() - yield mysql_node + mysql_8_0_node.start_and_wait() + yield mysql_8_0_node finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) + mysql_8_0_node.close() + run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_8_0_docker_compose, 'down', '--volumes', '--remove-orphans']) @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) @@ -146,11 +152,13 @@ def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") @@ -163,6 +171,7 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") @@ -179,10 +188,12 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") @@ -217,52 +228,64 @@ def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_m def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_tables_table(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") From 517ee6f142bbe46ebbabec11a865afa0a99eb41e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Mar 2021 10:45:51 +0300 Subject: [PATCH 114/266] rename --- tests/jepsen.clickhouse-keeper/doc/intro.md | 2 +- tests/jepsen.clickhouse-keeper/project.clj | 2 +- tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/doc/intro.md b/tests/jepsen.clickhouse-keeper/doc/intro.md index c6e5ccbd04a..09ce235c467 100644 --- a/tests/jepsen.clickhouse-keeper/doc/intro.md +++ b/tests/jepsen.clickhouse-keeper/doc/intro.md @@ -1,3 +1,3 @@ -# Introduction to jepsen.nukeeper +# Introduction to jepsen.keeper TODO: write [great documentation](http://jacobian.org/writing/what-to-write/) diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse-keeper/project.clj index c9b24e0ce2c..c38767a767d 100644 --- a/tests/jepsen.clickhouse-keeper/project.clj +++ b/tests/jepsen.clickhouse-keeper/project.clj @@ -1,4 +1,4 @@ -(defproject jepsen.nukeeper "0.1.0-SNAPSHOT" +(defproject jepsen.keeper "0.1.0-SNAPSHOT" :injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))] :description "A jepsen tests for ClickHouse Keeper" :url "https://clickhouse.tech/" diff --git a/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj b/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj index 35b1bab7b38..25333605351 100644 --- a/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj +++ b/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj @@ -15,7 +15,7 @@ (dorun (map (fn [v] (zk/delete conn v)) (take 10 (zk-range))))) (deftest a-test - (testing "nukeeper connection" + (testing "keeper connection" (.setLevel (LoggerFactory/getLogger "org.apache.zookeeper") Level/OFF) (let [conn (zk/connect "localhost:9181" :timeout-msec 5000)] diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 74b5a6ddf93..11db6fc61bc 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -58,7 +58,7 @@ int main(int argc, char *argv[]) Poco::Logger::root().setChannel(channel); Poco::Logger::root().setLevel("trace"); } - auto * logger = &Poco::Logger::get("nukeeper-dumper"); + auto * logger = &Poco::Logger::get("keeper-dumper"); ResponsesQueue queue; SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); From 501ee79a28117c6b62a7d01516d670fd100c5326 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 30 Mar 2021 11:49:54 +0300 Subject: [PATCH 115/266] small fixes --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3421afee15d..25d504aa939 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1874,7 +1874,7 @@ Default value: `0`. Enables or disables random shard insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table when there is no distributed key. -By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. +By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will reject any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 7509cb3ddb8..31ca598cda4 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1764,12 +1764,10 @@ ClickHouse генерирует исключение Включает или отключает режим вставки данных в [Distributed](../../engines/table-engines/special/distributed.md#distributed)) таблицу в случайный шард при отсутствии ключ шардирования. -По умолчанию при вставке данных в `Distributed` таблицу с несколькими шардами и при отсутствии ключа шардирования сервер ClickHouse будет отклонять любой запрос на вставку данных. Когда `insert_distributed_one_random_shard = 1`, добавления принимаются, а данные записываются в случайный шард. - +По умолчанию при вставке данных в `Distributed` таблицу с несколькими шардами и при отсутствии ключа шардирования сервер ClickHouse будет отклонять любой запрос на вставку данных. Когда `insert_distributed_one_random_shard = 1`, вставки принимаются, а данные записываются в случайный шард. Возможные значения: - - 0 — если у таблицы несколько шардов, но ключ шардирования отсутствует, вставка данных отклоняется. - 1 — если ключ шардирования отсутствует, то вставка данных осуществляется в случайный шард среди всех доступных шардов. From 676ea1efc3903103056defdadee8ce654ea9e3c1 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 11:52:38 +0300 Subject: [PATCH 116/266] The translate to Russian --- docs/en/sql-reference/functions/files.md | 2 +- docs/ru/sql-reference/functions/files.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 0e30d1bdd07..4cf1141e1ad 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: Files ## file {#file} -Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. +Reads file as a String. The file can contain subqueries, conditions, names of columns and any other information that will be read as one line. **Syntax** diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index 7252a3a18e2..8db29e0c4e6 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -1,13 +1,13 @@ --- toc_priority: 43 -toc_title: Files +toc_title: "Функции для работы с файлами" --- # Функции для работы с файлами {#funktsii-dlia-raboty-s-failami} ## file {#file} -Читает файл как строку. Файл может содержать подзапросы, условие, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. +Читает файл как строку. Файл может содержать подзапросы, условия, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. **Синтаксис** From 0bb369d5e2935b76e30b71acda6fd13ceb88cd07 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 30 Mar 2021 13:21:46 +0300 Subject: [PATCH 117/266] Code review fixes. --- src/Disks/S3/DiskS3.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 74c859efb75..211a9a63a74 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1030,7 +1030,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate directory {} to restorable schema", metadata_path + path); bool dir_contains_only_files = true; - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) if (isDirectory(it->path())) { dir_contains_only_files = false; @@ -1042,7 +1042,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r { auto result = getExecutor().execute([this, path] { - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) migrateFileToRestorableSchema(it->path()); }); @@ -1050,7 +1050,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r } else { - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) if (!isDirectory(it->path())) { auto source_path = it->path(); From 91035081e73523eca3f0d816e1b70f220d4f8618 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 13:54:49 +0300 Subject: [PATCH 118/266] Changes to the text --- .../server-configuration-parameters/settings.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 141a9a6b012..0b45488ebf7 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -502,7 +502,10 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa ## max_concurrent_queries {#max-concurrent-queries} -The maximum number of simultaneously processed requests. +The maximum number of simultaneously processed queries related to MergeTree table. Queries may be limited by other settings: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). + +!!! info "Note" + These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. Possible values: @@ -537,7 +540,7 @@ Default value: `0` that means no limit. ## min_marks_to_honor_max_concurrent_queries {#min-marks-to-honor-max-concurrent-queries} -Minimal number of marks for applying the `max_concurrent_queries` setting. +The minimal number of marks read by the query for applying the [max_concurrent_queries](#max-concurrent-queries) setting. Possible values: @@ -550,10 +553,6 @@ Possible values: 10 ``` -**See Also** - -- [max_concurrent_queries](#max-concurrent-queries) - ## max_connections {#max-connections} The maximum number of inbound connections. From 27635fd3813a8dc1d3fab0bde8a98118462a72ab Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 30 Mar 2021 14:03:00 +0300 Subject: [PATCH 119/266] create two batches per process in clickhouse-test --- tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 57f91b45b35..357be57ed6e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -788,7 +788,8 @@ def main(args): if jobs > run_total: run_total = jobs - batch_size = len(parallel_tests) // jobs + # Create two batches per process for more uniform execution time. + batch_size = max(1, len(parallel_tests) // (jobs * 2)) parallel_tests_array = [] for i in range(0, len(parallel_tests), batch_size): parallel_tests_array.append((parallel_tests[i:i+batch_size], suite, suite_dir, suite_tmp_dir)) From 873a98b33bb41497265dda4cee7990c4f71688c1 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 30 Mar 2021 16:19:05 +0300 Subject: [PATCH 120/266] Add test --- src/Core/Settings.h | 4 ++-- .../queries/0_stateless/01753_max_uri_size.reference | 1 + tests/queries/0_stateless/01753_max_uri_size.sh | 11 +++++++++++ 3 files changed, 14 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01753_max_uri_size.reference create mode 100755 tests/queries/0_stateless/01753_max_uri_size.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7ec7a87dfbc..f71a0cdc5dd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -227,7 +227,7 @@ class IColumn; M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ - M(UInt64, http_max_uri_size, 16384, "HTTP max URI length", 0) \ + M(UInt64, http_max_uri_size, 16384, "Maximum URI length of HTTP request", 0) \ M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \ @@ -538,7 +538,7 @@ struct Settings : public BaseSettings { /// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14. /// http://en.cppreference.com/w/cpp/language/aggregate_initialization - Settings() {} + Settings() = default; /** Set multiple settings from "profile" (in server configuration file (users.xml), profiles contain groups of multiple settings). * The profile can also be set using the `set` functions, like the profile setting. diff --git a/tests/queries/0_stateless/01753_max_uri_size.reference b/tests/queries/0_stateless/01753_max_uri_size.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01753_max_uri_size.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01753_max_uri_size.sh b/tests/queries/0_stateless/01753_max_uri_size.sh new file mode 100755 index 00000000000..7bd721cb63d --- /dev/null +++ b/tests/queries/0_stateless/01753_max_uri_size.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# NOTE: since 'max_uri_size' doesn't affect the request itself, this test hardly depends on the default value of this setting (16Kb). + +LONG_REQUEST=$(python3 -c "print('&max_uri_size=1'*2000, end='')") # ~30K + +${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}${LONG_REQUEST}&query=SELECT+1" 2>&1 | fgrep -c "HTTP/1.1 400 Bad Request" From fe06752727c2a78f604021e6491581d5c6a80979 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 30 Mar 2021 16:53:15 +0300 Subject: [PATCH 121/266] Update 01753_max_uri_size.sh --- tests/queries/0_stateless/01753_max_uri_size.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01753_max_uri_size.sh b/tests/queries/0_stateless/01753_max_uri_size.sh index 7bd721cb63d..5c63d9274fd 100755 --- a/tests/queries/0_stateless/01753_max_uri_size.sh +++ b/tests/queries/0_stateless/01753_max_uri_size.sh @@ -8,4 +8,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) LONG_REQUEST=$(python3 -c "print('&max_uri_size=1'*2000, end='')") # ~30K -${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}${LONG_REQUEST}&query=SELECT+1" 2>&1 | fgrep -c "HTTP/1.1 400 Bad Request" +${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}${LONG_REQUEST}&query=SELECT+1" 2>&1 | grep -Fc "HTTP/1.1 400 Bad Request" From 2162a19d35b5ae9d91b7dae0109103a18c92b151 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 30 Mar 2021 18:13:25 +0300 Subject: [PATCH 122/266] Fix flaky tests. --- tests/integration/test_quota/test.py | 12 +++++-- tests/integration/test_row_policy/test.py | 41 +++++++++++------------ 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 4374f46a39f..0f59ae27583 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -372,6 +372,7 @@ def test_dcl_management(): def test_users_xml_is_readonly(): assert re.search("storage is readonly", instance.query_and_get_error("DROP QUOTA myQuota")) + def test_query_inserts(): check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) @@ -380,9 +381,16 @@ def test_query_inserts(): system_quotas_usage( [["myQuota", "default", 1, 31556952, 0, 1000, 0, 500, 0, 500, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) - instance.query("INSERT INTO test_table values(1)") + instance.query("DROP TABLE IF EXISTS test_table_ins") + instance.query("CREATE TABLE test_table_ins(x UInt32) ENGINE = MergeTree ORDER BY tuple()") system_quota_usage( - [["myQuota", "default", 31556952, 1, 1000, 0, 500, 1, 500, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) + [["myQuota", "default", 31556952, 2, 1000, 0, 500, 0, 500, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) + + instance.query("INSERT INTO test_table_ins values(1)") + system_quota_usage( + [["myQuota", "default", 31556952, 3, 1000, 0, 500, 1, 500, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) + instance.query("DROP TABLE test_table_ins") + def test_consumption_of_show_tables(): assert instance.query("SHOW TABLES") == "test_table\n" diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index ffb6dcb0588..76aef325bfa 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -409,32 +409,31 @@ def test_tags_with_db_and_table_names(): def test_miscellaneous_engines(): - copy_policy_xml('normal_filters.xml') + node.query("CREATE ROW POLICY OR REPLACE pC ON mydb.other_table FOR SELECT USING a = 1 TO default") + assert node.query("SHOW ROW POLICIES ON mydb.other_table") == "pC\n" # ReplicatedMergeTree - node.query("DROP TABLE mydb.filtered_table1") - node.query( - "CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/00-00/filtered_table1', 'replica1') ORDER BY a") - node.query("INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1)") - assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) + node.query("DROP TABLE IF EXISTS mydb.other_table") + node.query("CREATE TABLE mydb.other_table (a UInt8, b UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/00-00/filtered_table1', 'replica1') ORDER BY a") + node.query("INSERT INTO mydb.other_table values (0, 0), (0, 1), (1, 0), (1, 1)") + assert node.query("SELECT * FROM mydb.other_table") == TSV([[1, 0], [1, 1]]) # CollapsingMergeTree - node.query("DROP TABLE mydb.filtered_table1") - node.query("CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE CollapsingMergeTree(b) ORDER BY a") - node.query("INSERT INTO mydb.filtered_table1 values (0, 1), (0, 1), (1, 1), (1, 1)") - assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 1], [1, 1]]) + node.query("DROP TABLE mydb.other_table") + node.query("CREATE TABLE mydb.other_table (a UInt8, b Int8) ENGINE CollapsingMergeTree(b) ORDER BY a") + node.query("INSERT INTO mydb.other_table values (0, 1), (0, 1), (1, 1), (1, 1)") + assert node.query("SELECT * FROM mydb.other_table") == TSV([[1, 1], [1, 1]]) # ReplicatedCollapsingMergeTree - node.query("DROP TABLE mydb.filtered_table1") - node.query( - "CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-01/filtered_table1', 'replica1', b) ORDER BY a") - node.query("INSERT INTO mydb.filtered_table1 values (0, 1), (0, 1), (1, 1), (1, 1)") - assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 1], [1, 1]]) + node.query("DROP TABLE mydb.other_table") + node.query("CREATE TABLE mydb.other_table (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-01/filtered_table1', 'replica1', b) ORDER BY a") + node.query("INSERT INTO mydb.other_table values (0, 1), (0, 1), (1, 1), (1, 1)") + assert node.query("SELECT * FROM mydb.other_table") == TSV([[1, 1], [1, 1]]) + + node.query("DROP ROW POLICY pC ON mydb.other_table") # DistributedMergeTree - node.query("DROP TABLE IF EXISTS mydb.not_filtered_table") - node.query( - "CREATE TABLE mydb.not_filtered_table (a UInt8, b UInt8) ENGINE Distributed('test_local_cluster', mydb, local)") - assert node.query("SELECT * FROM mydb.not_filtered_table", user="another") == TSV([[1, 0], [1, 1], [1, 0], [1, 1]]) - assert node.query("SELECT sum(a), b FROM mydb.not_filtered_table GROUP BY b ORDER BY b", user="another") == TSV( - [[2, 0], [2, 1]]) + node.query("DROP TABLE IF EXISTS mydb.other_table") + node.query("CREATE TABLE mydb.other_table (a UInt8, b UInt8) ENGINE Distributed('test_local_cluster', mydb, local)") + assert node.query("SELECT * FROM mydb.other_table", user="another") == TSV([[1, 0], [1, 1], [1, 0], [1, 1]]) + assert node.query("SELECT sum(a), b FROM mydb.other_table GROUP BY b ORDER BY b", user="another") == TSV([[2, 0], [2, 1]]) From 4f896d2b076c78b5ee2d030e643082923f287441 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Mar 2021 19:46:23 +0300 Subject: [PATCH 123/266] fix tests --- .../MergeTree/registerStorageMergeTree.cpp | 30 +++++++++++++++++++ ...ted_minimalistic_part_header_zookeeper.sql | 8 ++--- ...icated_mutations_empty_partition.reference | 2 +- ...6_replicated_mutations_empty_partition.sql | 6 ++-- ...tations_kill_many_replicas_long.reference} | 0 ...lter_mutations_kill_many_replicas_long.sh} | 0 6 files changed, 38 insertions(+), 8 deletions(-) rename tests/queries/0_stateless/{01593_concurrent_alter_mutations_kill_many_replicas.reference => 01593_concurrent_alter_mutations_kill_many_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01593_concurrent_alter_mutations_kill_many_replicas.sh => 01593_concurrent_alter_mutations_kill_many_replicas_long.sh} (100%) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6cde5245735..6dd005736f0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -20,6 +20,7 @@ #include #include +#include namespace DB @@ -410,6 +411,35 @@ static StoragePtr create(const StorageFactory::Arguments & args) throw Exception(msg, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } + if (is_extended_storage_def) + { + /// Allow expressions in engine arguments. + /// In new syntax argument can be literal or identifier or array/tuple of identifiers. + size_t arg_idx = 0; + try + { + for (; arg_idx < engine_args.size(); ++arg_idx) + { + auto & arg = engine_args[arg_idx]; + auto * arg_func = arg->as(); + if (!arg_func) + continue; + + /// If we got ASTFunction, let's evaluate it and replace with ASTLiteral. + /// Do not try evaluate array or tuple, because it's array or tuple of column identifiers. + if (arg_func->name == "array" || arg_func->name == "tuple") + continue; + Field value = evaluateConstantExpression(arg, args.local_context).first; + arg = std::make_shared(value); + } + } + catch (Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot evaluate engine argument {}: {} {}", + arg_idx, e.message(), getMergeTreeVerboseHelp(is_extended_storage_def)); + } + } + /// For Replicated. String zookeeper_path; String replica_name; diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 163ebf1af23..63897e225ce 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '1{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '1{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; CREATE TABLE part_header_r2(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '2{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, @@ -39,10 +39,10 @@ SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference index 1f7146dfe48..2f204867c41 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference @@ -2,4 +2,4 @@ 10 10 24 -CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/default/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index 63a3069c518..73245fe49ec 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -5,7 +5,7 @@ CREATE TABLE replicated_mutations_empty_partitions key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') ORDER BY key PARTITION by key; @@ -13,7 +13,7 @@ INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number SELECT count(distinct value) FROM replicated_mutations_empty_partitions; -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4'; @@ -21,7 +21,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9'; -- still ten records -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference rename to tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh similarity index 100% rename from tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh rename to tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh From fb8fdf7aa1ce8ef0b54a4af1d8cca4189cd4a4c3 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:10:03 +0300 Subject: [PATCH 124/266] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 4cf1141e1ad..c081daeae7c 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: Files ## file {#file} -Reads file as a String. The file can contain subqueries, conditions, names of columns and any other information that will be read as one line. +Reads file as a String. The file content is not parsed, so any information is read as one string and placed into the specified column. **Syntax** @@ -33,4 +33,3 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt'); - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) - [file](../table-functions/file.md) - From 3573a2ba4bfc397a5879c048c80b1d9df1233a2b Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:10:15 +0300 Subject: [PATCH 125/266] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index c081daeae7c..5bc84649c89 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** From cde71fbbe5eb839f710100f858a1f06f46b037dd Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:11:30 +0300 Subject: [PATCH 126/266] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 5bc84649c89..9cbf8932465 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -21,7 +21,7 @@ file(path) **Example** -Inserting data from files a.txt and b.txt into a table as different rows. +Inserting data from files a.txt and b.txt into a table as strings: Query: From e74e3df04ab1509246dfade37b9c28df30a862da Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:12:39 +0300 Subject: [PATCH 127/266] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index 8db29e0c4e6..d4152a52b31 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы с файлами" ## file {#file} -Читает файл как строку. Файл может содержать подзапросы, условия, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. +Читает файл как строку. Содержимое файла не разбирается (не парсится) и записывается в указанную колонку в виде единой строки. **Синтаксис** @@ -31,4 +31,3 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt'); - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) - [file](../table-functions/file.md) - From 3676f5d7983dd43d33344dbb2904e440613038c7 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:12:57 +0300 Subject: [PATCH 128/266] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index d4152a52b31..d7792da1929 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -21,7 +21,7 @@ file(path) **Примеры** -Вставка данных из файлов a.txt и b.txt в таблицу в виде отдельных строк: +Вставка данных из файлов a.txt и b.txt в таблицу в виде строк: ``` sql INSERT INTO table SELECT file('a.txt'), file('b.txt'); From 049019bdb21df5d25a6a2239903fdba8167d9009 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 30 Mar 2021 20:27:58 +0300 Subject: [PATCH 129/266] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b7b72e0de41..afb02e7ea0b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -227,7 +227,7 @@ def get_processlist(args): query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ b"FROM clusterAllReplicas('r', system.processes) WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=20) + (stdout, _) = clickhouse_proc.communicate((query), timeout=20) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) From 2ab2de6fd3bb30d2eb05b2fb456ce55fa1d6ae62 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:34:21 +0300 Subject: [PATCH 130/266] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index d7792da1929..9cb659375b9 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -17,7 +17,7 @@ file(path) **Аргументы** -- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. +- `path` — относительный путь до файла от [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Путь к файлу может включать следующие символы подстановки и шаблоны: `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. **Примеры** From 6690a2e043f0457680c8e7538783d462c32e6479 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Tue, 30 Mar 2021 20:35:38 +0300 Subject: [PATCH 131/266] Last changes restored --- .../table-engines/integrations/postgresql.md | 39 ++++++---- .../table-functions/postgresql.md | 45 ++++++++---- .../table-engines/integrations/postgresql.md | 49 ++++++++----- .../table-functions/postgresql.md | 71 ++++++++++++------- 4 files changed, 134 insertions(+), 70 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 3560ad71a22..7bf930d4120 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -15,7 +15,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], ... -) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); +) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'[, `schema`]); ``` See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. @@ -29,27 +29,28 @@ The table structure can differ from the original PostgreSQL table structure: **Engine Parameters** - `host:port` — PostgreSQL server address. - - `database` — Remote database name. - - `table` — Remote table name. - - `user` — PostgreSQL user. - - `password` — User password. +- `schema` — Non-default table schema. Optional. + +## Implementation Details {#implementation-details} `SELECT` queries on PostgreSQL side run as `COPY (SELECT ...) TO STDOUT` inside read-only PostgreSQL transaction with commit after each `SELECT` query. -Simple `WHERE` clauses such as `=, !=, >, >=, <, <=, IN` are executed on the PostgreSQL server. +Simple `WHERE` clauses such as `=`, `!=`, `>`, `>=`, `<`, `<=`, and `IN` are executed on the PostgreSQL server. All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` sampling constraint are executed in ClickHouse only after the query to PostgreSQL finishes. `INSERT` queries on PostgreSQL side run as `COPY "table_name" (field1, field2, ... fieldN) FROM STDIN` inside PostgreSQL transaction with auto-commit after each `INSERT` statement. -PostgreSQL Array types converts into ClickHouse arrays. +PostgreSQL `Array` types are converted into ClickHouse arrays. !!! info "Note" - Be careful - in PostgreSQL an array data created like a `type_name[]` may contain multi-dimensional arrays of different dimensions in different table rows in same column, but in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. + Be careful - in PostgreSQL an array data, created like a `type_name[]`, may contain multi-dimensional arrays of different dimensions in different table rows in same column. But in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. + +Supports replicas priority for PostgreSQL dictionary source. The bigger the number in map, the less the priority. The highest priority is `0`. ## Usage Example {#usage-example} @@ -66,10 +67,10 @@ PRIMARY KEY (int_id)); CREATE TABLE -postgres=# insert into test (int_id, str, "float") VALUES (1,'test',2); +postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2); INSERT 0 1 -postgresql> select * from test; +postgresql> SELECT * FROM test; int_id | int_nullable | float | str | float_nullable --------+--------------+-------+------+---------------- 1 | | 2 | test | @@ -89,14 +90,28 @@ ENGINE = PostgreSQL('localhost:5432', 'public', 'test', 'postges_user', 'postgre ``` ``` sql -SELECT * FROM postgresql_table WHERE str IN ('test') +SELECT * FROM postgresql_table WHERE str IN ('test'); ``` ``` text ┌─float_nullable─┬─str──┬─int_id─┐ │ ᴺᵁᴸᴸ │ test │ 1 │ └────────────────┴──────┴────────┘ -1 rows in set. Elapsed: 0.019 sec. +``` + +Using Non-default Schema: + +```text +postgres=# CREATE SCHEMA "nice.schema"; + +postgres=# CREATE TABLE "nice.schema"."nice.table" (a integer); + +postgres=# INSERT INTO "nice.schema"."nice.table" SELECT i FROM generate_series(0, 99) as t(i) +``` + +```sql +CREATE TABLE pg_table_schema_with_dots (a UInt32) + ENGINE PostgreSQL('localhost:5432', 'clickhouse', 'nice.table', 'postgrsql_user', 'password', 'nice.schema'); ``` **See Also** diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index b6f6b8bf5cb..bfb5fdf9be6 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -10,25 +10,30 @@ Allows `SELECT` and `INSERT` queries to be performed on data that is stored on a **Syntax** ``` sql -postgresql('host:port', 'database', 'table', 'user', 'password') +postgresql('host:port', 'database', 'table', 'user', 'password'[, `schema`]) ``` **Arguments** - `host:port` — PostgreSQL server address. - - `database` — Remote database name. - - `table` — Remote table name. - - `user` — PostgreSQL user. - - `password` — User password. +- `schema` — Non-default table schema. Optional. +**Returned Value** + +A table object with the same columns as the original PostgreSQL table. + +!!! info "Note" + In the `INSERT` query to distinguish table function `postgresql(...)` from table name with column names list you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. + +## Implementation Details {#implementation-details} `SELECT` queries on PostgreSQL side run as `COPY (SELECT ...) TO STDOUT` inside read-only PostgreSQL transaction with commit after each `SELECT` query. -Simple `WHERE` clauses such as `=, !=, >, >=, <, <=, IN` are executed on the PostgreSQL server. +Simple `WHERE` clauses such as `=`, `!=`, `>`, `>=`, `<`, `<=`, and `IN` are executed on the PostgreSQL server. All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` sampling constraint are executed in ClickHouse only after the query to PostgreSQL finishes. @@ -37,14 +42,9 @@ All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` samp PostgreSQL Array types converts into ClickHouse arrays. !!! info "Note" - Be careful in PostgreSQL an array data type column like Integer[] may contain arrays of different dimensions in different rows, but in ClickHouse it is only allowed to have multidimensional arrays of the same dimension in all rows. + Be careful, in PostgreSQL an array data type column like Integer[] may contain arrays of different dimensions in different rows, but in ClickHouse it is only allowed to have multidimensional arrays of the same dimension in all rows. -**Returned Value** - -A table object with the same columns as the original PostgreSQL table. - -!!! info "Note" - In the `INSERT` query to distinguish table function `postgresql(...)` from table name with column names list you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. +Supports replicas priority for PostgreSQL dictionary source. The bigger the number in map, the less the priority. The highest priority is `0`. **Examples** @@ -61,10 +61,10 @@ PRIMARY KEY (int_id)); CREATE TABLE -postgres=# insert into test (int_id, str, "float") VALUES (1,'test',2); +postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2); INSERT 0 1 -postgresql> select * from test; +postgresql> SELECT * FROM test; int_id | int_nullable | float | str | float_nullable --------+--------------+-------+------+---------------- 1 | | 2 | test | @@ -97,6 +97,21 @@ SELECT * FROM postgresql('localhost:5432', 'test', 'test', 'postgresql_user', 'p └────────┴──────────────┴───────┴──────┴────────────────┘ ``` +Using Non-default Schema: + +```text +postgres=# CREATE SCHEMA "nice.schema"; + +postgres=# CREATE TABLE "nice.schema"."nice.table" (a integer); + +postgres=# INSERT INTO "nice.schema"."nice.table" SELECT i FROM generate_series(0, 99) as t(i) +``` + +```sql +CREATE TABLE pg_table_schema_with_dots (a UInt32) + ENGINE PostgreSQL('localhost:5432', 'clickhouse', 'nice.table', 'postgrsql_user', 'password', 'nice.schema'); +``` + **See Also** - [The PostgreSQL table engine](../../engines/table-engines/integrations/postgresql.md) diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md index 62faa3da54f..919dbfe625b 100644 --- a/docs/ru/engines/table-engines/integrations/postgresql.md +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -1,4 +1,4 @@ ---- +--- toc_priority: 11 toc_title: PostgreSQL --- @@ -15,7 +15,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], ... -) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'); +) ENGINE = PostgreSQL('host:port', 'database', 'table', 'user', 'password'[, `schema`]); ``` Смотрите подробное описание запроса [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query). @@ -29,28 +29,28 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **Параметры движка** - `host:port` — адрес сервера PostgreSQL. - - `database` — Имя базы данных на сервере PostgreSQL. - - `table` — Имя таблицы. - - `user` — Имя пользователя PostgreSQL. - - `password` — Пароль пользователя PostgreSQL. +- `schema` — имя схемы, если не используется схема по умолчанию. Необязательный аргумент. -`SELECT` запросы на стороне PostgreSQL выполняются как `COPY (SELECT ...) TO STDOUT` внутри транзакции PostgreSQL только на чтение с коммитом после каждого `SELECT` запроса. +## Особенности реализации {#implementation-details} -Простые условия для `WHERE` такие как `=, !=, >, >=, <, <=, IN` исполняются на стороне PostgreSQL сервера. +Запросы `SELECT` на стороне PostgreSQL выполняются как `COPY (SELECT ...) TO STDOUT` внутри транзакции PostgreSQL только на чтение с коммитом после каждого запроса `SELECT`. -Все операции объединения, аггрегации, сортировки, условия `IN [ array ]` и ограничения `LIMIT` выполняются на стороне ClickHouse только после того как запрос к PostgreSQL закончился. +Простые условия для `WHERE`, такие как `=`, `!=`, `>`, `>=`, `<`, `<=` и `IN`, исполняются на стороне PostgreSQL сервера. -`INSERT` запросы на стороне PostgreSQL выполняются как `COPY "table_name" (field1, field2, ... fieldN) FROM STDIN` внутри PostgreSQL транзакции с автоматическим коммитом после каждого `INSERT` запроса. +Все операции объединения, аггрегации, сортировки, условия `IN [ array ]` и ограничения `LIMIT` выполняются на стороне ClickHouse только после того, как запрос к PostgreSQL закончился. + +Запросы `INSERT` на стороне PostgreSQL выполняются как `COPY "table_name" (field1, field2, ... fieldN) FROM STDIN` внутри PostgreSQL транзакции с автоматическим коммитом после каждого запроса `INSERT`. PostgreSQL массивы конвертируются в массивы ClickHouse. !!! info "Внимание" - Будьте осторожны в PostgreSQL массивы созданные как `type_name[]`, являются многомерными и могут содержать в себе разное количество измерений в разных строках одной таблицы, внутри ClickHouse допустипы только многомерные массивы с одинаковым кол-вом измерений во всех строках таблицы. + Будьте внимательны, в PostgreSQL массивы, созданные как `type_name[]`, являются многомерными и могут содержать в себе разное количество измерений в разных строках одной таблицы. Внутри ClickHouse допустимы только многомерные массивы с одинаковым кол-вом измерений во всех строках таблицы. +При использовании словаря PostgreSQL поддерживается приоритет реплик. Чем больше номер реплики, тем ниже ее приоритет. Наивысший приоритет у реплики с номером `0`. ## Пример использования {#usage-example} Таблица в PostgreSQL: @@ -66,10 +66,10 @@ PRIMARY KEY (int_id)); CREATE TABLE -postgres=# insert into test (int_id, str, "float") VALUES (1,'test',2); +postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2); INSERT 0 1 -postgresql> select * from test; +postgresql> SELECT * FROM test; int_id | int_nullable | float | str | float_nullable --------+--------------+-------+------+---------------- 1 | | 2 | test | @@ -89,18 +89,33 @@ ENGINE = PostgreSQL('localhost:5432', 'public', 'test', 'postges_user', 'postgre ``` ``` sql -SELECT * FROM postgresql_table WHERE str IN ('test') +SELECT * FROM postgresql_table WHERE str IN ('test'); ``` ``` text ┌─float_nullable─┬─str──┬─int_id─┐ │ ᴺᵁᴸᴸ │ test │ 1 │ └────────────────┴──────┴────────┘ -1 rows in set. Elapsed: 0.019 sec. ``` -**Смотри также** {#see-also}** +Using Non-default Schema: + +```text +postgres=# CREATE SCHEMA "nice.schema"; + +postgres=# CREATE TABLE "nice.schema"."nice.table" (a integer); + +postgres=# INSERT INTO "nice.schema"."nice.table" SELECT i FROM generate_series(0, 99) as t(i) +``` + +```sql +CREATE TABLE pg_table_schema_with_dots (a UInt32) + ENGINE PostgreSQL('localhost:5432', 'clickhouse', 'nice.table', 'postgrsql_user', 'password', 'nice.schema'); +``` + +**См. также** - [Табличная функция `postgresql`](../../../sql-reference/table-functions/postgresql.md) -- [Использование PostgreSQL в качестве истояника для внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [Использование PostgreSQL в качестве источника для внешнего словаря](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +[Оригинальная статья](https://clickhouse.tech/docs/ru/engines/table-engines/integrations/postgresql/) diff --git a/docs/ru/sql-reference/table-functions/postgresql.md b/docs/ru/sql-reference/table-functions/postgresql.md index 142c10c8736..66637276726 100644 --- a/docs/ru/sql-reference/table-functions/postgresql.md +++ b/docs/ru/sql-reference/table-functions/postgresql.md @@ -5,44 +5,46 @@ toc_title: postgresql # postgresql {#postgresql} -Позволяет выполнять заполсы `SELECT` и `INSERT` над таблицами удаленной БД PostgreSQL. +Позволяет выполнять запросы `SELECT` и `INSERT` над таблицами удаленной БД PostgreSQL. **Синтаксис** ``` sql -postgresql('host:port', 'database', 'table', 'user', 'password') +postgresql('host:port', 'database', 'table', 'user', 'password'[, `schema`]) ``` -**Параметры** +**Аргументы** - `host:port` — адрес сервера PostgreSQL. - - `database` — имя базы данных на удалённом сервере. - - `table` — имя таблицы на удалённом сервере. - - `user` — пользователь PostgreSQL. - - `password` — пароль пользователя. - - -SELECT запросы на стороне PostgreSQL выполняются как `COPY (SELECT ...) TO STDOUT` внутри транзакции PostgreSQL только на чтение с коммитом после каждого `SELECT` запроса. - -Простые условия для `WHERE` такие как `=, !=, >, >=, <, <=, IN` исполняются на стороне PostgreSQL сервера. - -Все операции объединения, аггрегации, сортировки, условия `IN [ array ]` и ограничения `LIMIT` выполняются на стороне ClickHouse только после того как запрос к PostgreSQL закончился. - -INSERT запросы на стороне PostgreSQL выполняются как `COPY "table_name" (field1, field2, ... fieldN) FROM STDIN` внутри PostgreSQL транзакции с автоматическим коммитом после каждого `INSERT` запроса. - -PostgreSQL массивы конвертируются в массивы ClickHouse. -Будьте осторожны в PostgreSQL массивы созданные как type_name[], являются многомерными и могут содержать в себе разное количество измерений в разных строках одной таблицы, внутри ClickHouse допустипы только многомерные массивы с одинаковым кол-вом измерений во всех строках таблицы. +- `schema` — имя схемы, если не используется схема по умолчанию. Необязательный аргумент. **Возвращаемое значение** -Объект таблицы с теми же столбцами, что и в исходной таблице PostgreSQL. +Таблица с теми же столбцами, что и в исходной таблице PostgreSQL. !!! info "Примечание" -В запросах `INSERT` для того чтобы отличить табличную функцию `postgresql(...)` от таблицы со списком имен столбцов вы должны указывать ключевые слова `FUNCTION` или `TABLE FUNCTION`. See examples below. + В запросах `INSERT` для того чтобы отличить табличную функцию `postgresql(...)` от таблицы со списком имен столбцов вы должны указывать ключевые слова `FUNCTION` или `TABLE FUNCTION`. См. примеры ниже. + +## Особенности реализации {#implementation-details} + +Запросы `SELECT` на стороне PostgreSQL выполняются как `COPY (SELECT ...) TO STDOUT` внутри транзакции PostgreSQL только на чтение с коммитом после каждого запроса `SELECT`. + +Простые условия для `WHERE` такие как `=`, `!=`, `>`, `>=`, `<`, `<=` и `IN` исполняются на стороне PostgreSQL сервера. + +Все операции объединения, аггрегации, сортировки, условия `IN [ array ]` и ограничения `LIMIT` выполняются на стороне ClickHouse только после того как запрос к PostgreSQL закончился. + +Запросы `INSERT` на стороне PostgreSQL выполняются как `COPY "table_name" (field1, field2, ... fieldN) FROM STDIN` внутри PostgreSQL транзакции с автоматическим коммитом после каждого запроса `INSERT`. + +PostgreSQL массивы конвертируются в массивы ClickHouse. + +!!! info "Примечание" + Будьте внимательны, в PostgreSQL массивы, созданные как `type_name[]`, являются многомерными и могут содержать в себе разное количество измерений в разных строках одной таблицы. Внутри ClickHouse допустипы только многомерные массивы с одинаковым кол-вом измерений во всех строках таблицы. + +При использовании словаря PostgreSQL поддерживается приоритет реплик. Чем больше номер реплики, тем ниже ее приоритет. Наивысший приоритет у реплики с номером `0`. **Примеры** @@ -59,10 +61,10 @@ PRIMARY KEY (int_id)); CREATE TABLE -postgres=# insert into test (int_id, str, "float") VALUES (1,'test',2); +postgres=# INSERT INTO test (int_id, str, "float") VALUES (1,'test',2); INSERT 0 1 -postgresql> select * from test; +postgresql> SELECT * FROM test; int_id | int_nullable | float | str | float_nullable --------+--------------+-------+------+---------------- 1 | | 2 | test | @@ -81,7 +83,7 @@ SELECT * FROM postgresql('localhost:5432', 'test', 'test', 'postgresql_user', 'p └────────┴──────────────┴───────┴──────┴────────────────┘ ``` -Вставка: +Вставка данных: ```sql INSERT INTO TABLE FUNCTION postgresql('localhost:5432', 'test', 'test', 'postgrsql_user', 'password') (int_id, float) VALUES (2, 3); @@ -95,7 +97,24 @@ SELECT * FROM postgresql('localhost:5432', 'test', 'test', 'postgresql_user', 'p └────────┴──────────────┴───────┴──────┴────────────────┘ ``` -**Смотрите также** +Using Non-default Schema: -- [Движок таблиц `PostgreSQL`](../../sql-reference/table-functions/postgresql.md) +```text +postgres=# CREATE SCHEMA "nice.schema"; + +postgres=# CREATE TABLE "nice.schema"."nice.table" (a integer); + +postgres=# INSERT INTO "nice.schema"."nice.table" SELECT i FROM generate_series(0, 99) as t(i) +``` + +```sql +CREATE TABLE pg_table_schema_with_dots (a UInt32) + ENGINE PostgreSQL('localhost:5432', 'clickhouse', 'nice.table', 'postgrsql_user', 'password', 'nice.schema'); +``` + +**См. также** + +- [Движок таблиц PostgreSQL](../../sql-reference/table-functions/postgresql.md) - [Использование PostgreSQL как источника данных для внешнего словаря](../../sql-reference/table-functions/postgresql.md#dicts-external_dicts_dict_sources-postgresql) + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/postgresql/) From d82d3a9e656d5baaee748d3ae7d3282dc6719c94 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Mar 2021 18:48:33 +0000 Subject: [PATCH 132/266] Try fix flaky rabbitmq test --- tests/integration/test_storage_rabbitmq/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index ca89ebdea0a..bb8da289490 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -258,6 +258,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'tsv', rabbitmq_format = 'TSV', + rabbitmq_queue_base = 'tsv', rabbitmq_row_delimiter = '\\n'; ''') From 0efb41d5611189de34567b0029d01ce05ad39393 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 22:06:03 +0300 Subject: [PATCH 133/266] translation into Russian --- .../settings.md | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index b50347f6196..09a88fd5c63 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -481,7 +481,15 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## max_concurrent_queries {#max-concurrent-queries} -Максимальное количество одновременно обрабатываемых запросов. +Максимальное количество одновременно обрабатываемых запросов, связанных к семейством таблиц `MergeTree`. Запросы могут быть ограничены настройками: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). + +!!! info "Примечание" + Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. + +Возможные значения: + +- Положительное целое число. +- 0 — выключена. **Пример** @@ -509,6 +517,21 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - [max_concurrent_queries](#max-concurrent-queries) +## min_marks_to_honor_max_concurrent_queries {#min-marks-to-honor-max-concurrent-queries} + +Минимальное количество меток, которое считает запрос, для применения настройки [max_concurrent_queries](#max-concurrent-queries). + +Возможные значения: + +- Положительное целое число. +- 0 — выключена. + +**Example** + +``` xml +10 +``` + ## max_connections {#max-connections} Максимальное количество входящих соединений. From 47431df7ce954d239daed0ee52f3d1d874a0819e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 30 Mar 2021 22:18:03 +0300 Subject: [PATCH 134/266] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bb8da289490..5ea3d19ebfe 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -275,7 +275,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): channel.basic_publish(exchange='tsv', routing_key='', body=message) connection.close() - time.sleep(1) + time.sleep(2) result = '' while True: From 3936c5650f14fc320f5c855a2a5684ba05c222f9 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 22:20:54 +0300 Subject: [PATCH 135/266] translation into Russian --- .../ru/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 09a88fd5c63..7dde8e609d5 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -481,7 +481,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## max_concurrent_queries {#max-concurrent-queries} -Максимальное количество одновременно обрабатываемых запросов, связанных к семейством таблиц `MergeTree`. Запросы могут быть ограничены настройками: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). +Определяет максимальное количество одновременно обрабатываемых запросов, связанных к семейством таблиц `MergeTree`. Запросы также могут быть ограничены настройками: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). !!! info "Примечание" Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. @@ -519,7 +519,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## min_marks_to_honor_max_concurrent_queries {#min-marks-to-honor-max-concurrent-queries} -Минимальное количество меток, которое считает запрос, для применения настройки [max_concurrent_queries](#max-concurrent-queries). +Определяет минимальное количество меток, которое считает запрос, для применения настройки [max_concurrent_queries](#max-concurrent-queries). Возможные значения: From 4917366ac6db807d1ad0b674e7837af3dc1174ed Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 22:35:47 +0300 Subject: [PATCH 136/266] Example --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 7dde8e609d5..6ef52614c21 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -526,7 +526,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - Положительное целое число. - 0 — выключена. -**Example** +**Пример** ``` xml 10 From a712b34eed2d0a7a9c39c085aac9be73ec9e4725 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 31 Mar 2021 00:06:43 +0300 Subject: [PATCH 137/266] Update 01593_concurrent_alter_mutations_kill_many_replicas_long.reference --- ...t_alter_mutations_kill_many_replicas_long.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference index 06e637bc736..f7c65e36be4 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference @@ -4,13 +4,13 @@ 499999500000 499999500000 Metadata version on replica 1 equal with first replica, OK -CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 2 equal with first replica, OK -CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 3 equal with first replica, OK -CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 4 equal with first replica, OK -CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 5 equal with first replica, OK -CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 From a59c66c26650479682a7fa3756ac50a1694f5e2f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 04:07:18 +0300 Subject: [PATCH 138/266] Fix UBSan report in mapOp --- src/Functions/array/mapOp.cpp | 12 +++++++++--- .../queries/0_stateless/01781_map_op_ubsan.reference | 1 + tests/queries/0_stateless/01781_map_op_ubsan.sql | 1 + 3 files changed, 11 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01781_map_op_ubsan.reference create mode 100644 tests/queries/0_stateless/01781_map_op_ubsan.sql diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index 62662099961..ec33f48f42b 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -120,6 +120,12 @@ private: return res; } + template + static inline auto NO_SANITIZE_UNDEFINED negate(T x) + { + return -x; + } + template ColumnPtr execute2(size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const { @@ -159,14 +165,14 @@ private: if constexpr (is_str_key) { // have to use Field structs to get strings - key = arg.key_column.operator[](offset + j).get(); + key = arg.key_column[offset + j].get(); } else { key = assert_cast &>(arg.key_column).getData()[offset + j]; } - auto value = arg.val_column.operator[](offset + j).get(); + ValType value = arg.val_column[offset + j].get(); if constexpr (op_type == OpTypes::ADD) { @@ -177,7 +183,7 @@ private: else { static_assert(op_type == OpTypes::SUBTRACT); - const auto [it, inserted] = summing_map.insert({key, first ? value : -value}); + const auto [it, inserted] = summing_map.insert({key, first ? value : negate(value)}); if (!inserted) it->second -= value; } diff --git a/tests/queries/0_stateless/01781_map_op_ubsan.reference b/tests/queries/0_stateless/01781_map_op_ubsan.reference new file mode 100644 index 00000000000..030c8bb5ab4 --- /dev/null +++ b/tests/queries/0_stateless/01781_map_op_ubsan.reference @@ -0,0 +1 @@ +\N (([0,10,255],[-9223372036854775808,1025,0]),[255,NULL]) \N ([0,255],3,[-2]) [NULL] diff --git a/tests/queries/0_stateless/01781_map_op_ubsan.sql b/tests/queries/0_stateless/01781_map_op_ubsan.sql new file mode 100644 index 00000000000..adbb5d5a8d7 --- /dev/null +++ b/tests/queries/0_stateless/01781_map_op_ubsan.sql @@ -0,0 +1 @@ +SELECT toInt32([toUInt8(NULL)], NULL), (mapSubtract(([toUInt8(256), 10], [toInt32(-9223372036854775808), 1025]), ([toUInt8(65535), 0], [toInt16(0.), -9223372036854775808])), [toUInt8(-1), toInt32(([toUInt8(9223372036854775807), -1], [toInt32(255), 65536]), NULL)]), toUInt8(([2, 9223372036854775807], [toFloat32('0.0000065536'), 2]), 9223372036854775807, NULL), ([toUInt8(1024), 255], toUInt8(3), [toInt16(-2)]), [NULL]; From 1b218465c5a2a1740ca21d955b8c907608d75e8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 04:41:25 +0300 Subject: [PATCH 139/266] Remove recursive submodules --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 744bdfe188f..616b3dc76a0 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 744bdfe188f018e5e05f5deebd4e9ee0a7706cf4 +Subproject commit 616b3dc76a0c8450b4027ded8a78e9619d7c845f From 7b1890f0b3e096a43006e4163b98f6316ac1e91b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 06:19:34 +0300 Subject: [PATCH 140/266] Fix some OOMs in stress tests --- src/Common/AllocatorWithMemoryTracking.h | 62 +++++++++++++++++++ src/Core/Field.h | 3 +- .../0_stateless/01782_field_oom.reference | 0 tests/queries/0_stateless/01782_field_oom.sql | 2 + 4 files changed, 66 insertions(+), 1 deletion(-) create mode 100644 src/Common/AllocatorWithMemoryTracking.h create mode 100644 tests/queries/0_stateless/01782_field_oom.reference create mode 100644 tests/queries/0_stateless/01782_field_oom.sql diff --git a/src/Common/AllocatorWithMemoryTracking.h b/src/Common/AllocatorWithMemoryTracking.h new file mode 100644 index 00000000000..e9597e4bf5a --- /dev/null +++ b/src/Common/AllocatorWithMemoryTracking.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include + +#include + + +/// Implementation of std::allocator interface that tracks memory with MemoryTracker. +/// NOTE We already plug MemoryTracker into new/delete operators. So, everything works even with default allocator. +/// But it is enabled only if jemalloc is used (to obtain the size of the allocation on call to delete). +/// And jemalloc is disabled for builds with sanitizers. In these cases memory was not always tracked. + +template +struct AllocatorWithMemoryTracking +{ + typedef T value_type; + + AllocatorWithMemoryTracking() = default; + + template + constexpr AllocatorWithMemoryTracking(const AllocatorWithMemoryTracking &) noexcept + { + } + + [[nodiscard]] T * allocate(size_t n) + { + if (n > std::numeric_limits::max() / sizeof(T)) + throw std::bad_alloc(); + + size_t bytes = n * sizeof(T); + CurrentMemoryTracker::alloc(bytes); + + T * p = static_cast(malloc(bytes)); + if (!p) + throw std::bad_alloc(); + + return p; + } + + void deallocate(T * p, size_t n) noexcept + { + free(p); + + size_t bytes = n * sizeof(T); + CurrentMemoryTracker::free(bytes); + } +}; + +template +bool operator==(const AllocatorWithMemoryTracking &, const AllocatorWithMemoryTracking &) +{ + return true; +} + +template +bool operator!=(const AllocatorWithMemoryTracking &, const AllocatorWithMemoryTracking &) +{ + return false; +} + diff --git a/src/Core/Field.h b/src/Core/Field.h index 81d06693a7f..77549854982 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -35,7 +36,7 @@ template using NearestFieldType = typename NearestFieldTypeImpl::Type; class Field; -using FieldVector = std::vector; +using FieldVector = std::vector>; /// Array and Tuple use the same storage type -- FieldVector, but we declare /// distinct types for them, so that the caller can choose whether it wants to diff --git a/tests/queries/0_stateless/01782_field_oom.reference b/tests/queries/0_stateless/01782_field_oom.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01782_field_oom.sql b/tests/queries/0_stateless/01782_field_oom.sql new file mode 100644 index 00000000000..60326b09a7a --- /dev/null +++ b/tests/queries/0_stateless/01782_field_oom.sql @@ -0,0 +1,2 @@ +SET max_memory_usage = '1G'; +SELECT sumMap([number], [number]) FROM system.numbers_mt; -- { serverError 241 } From fefe3e69e5b57ceab8e2ba4689b6377fb026ab81 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 31 Mar 2021 11:45:37 +0800 Subject: [PATCH 141/266] trigger CI again From 7709402422d56f2863a6ba360b38925755519cb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 08:15:58 +0300 Subject: [PATCH 142/266] Add more adopters --- docs/en/introduction/adopters.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 23f7b596851..4d84895afa3 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -15,6 +15,7 @@ toc_title: Adopters |
Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | +| ApiRoad | API marketplace | Analytics | — | — | [Blog post, Nov 2018, Mar 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | | Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | | ArenaData | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | | Avito | Classifieds | Monitoring | — | — | [Meetup, April 2020](https://www.youtube.com/watch?v=n1tm4j4W8ZQ) | @@ -37,6 +38,7 @@ toc_title: Adopters | CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | | Crazypanda | Games | | — | — | Live session on ClickHouse meetup | | Criteo | Retail | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) | +| Cryptology | Digital Assets Trading Platform | — | — | — | [Job advertisement, March 2021](https://career.habr.com/companies/cryptology/vacancies) | | Dataliance for China Telecom | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | | Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | | Deeplay | Gaming Analytics | — | — | — | [Job advertisement, 2020](https://career.habr.com/vacancies/1000062568) | @@ -49,6 +51,7 @@ toc_title: Adopters | FunCorp | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | +| Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | @@ -65,15 +68,18 @@ toc_title: Adopters | Lawrence Berkeley National Laboratory | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | | LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | +| MAXILECT | Ad Tech, Blockchain, ML, AI | — | — | — | [Job advertisement, 2021](https://www.linkedin.com/feed/update/urn:li:activity:6780842017229430784/) | | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | | Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | | MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x | MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | +| Netskope | Network Security | — | — | — | [Job advertisement, March 2021](https://www.mendeley.com/careers/job/senior-software-developer-backend-developer-1346348) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | | OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | +| OZON | E-commerce | — | — | — | [Official website](https://job.ozon.ru/vacancy/razrabotchik-clickhouse-ekspluatatsiya-40991870/) | | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | | Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | | Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | @@ -92,12 +98,14 @@ toc_title: Adopters | S7 Airlines | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | | scireum GmbH | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | | Segment | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | +| sembot.io | Shopping Ads | — | — | — | A comment on LinkedIn, 2020 | | SEMrush | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | | Sentry | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | | seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | | SGK | Goverment Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | | Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | | SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | +| Spark New Zealand | Telecommunications | Security Operations | — | — | [Blog Post, Feb 2020](https://blog.n0p.me/2020/02/2020-02-05-dnsmonster/) | | Splunk | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | | Spotify | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | | Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | @@ -106,13 +114,16 @@ toc_title: Adopters | Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | | Tencent Music Entertainment (TME) | BigData | Data processing | — | — | [Blog in Chinese, June 2020](https://cloud.tencent.com/developer/article/1637840) | +| Tinybird | Real-time Data Products | Data processing | — | — | [Official website](https://www.tinybird.co/) | | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | +| Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | | Wisebits | IT Solutions | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | | Workato | Automation Software | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=334) | +| Xenoss | Marketing, Advertising | — | — | — | [Instagram, March 2021](https://www.instagram.com/p/CNATV7qBgB1/) | | Xiaoxin Tech | Education | Common purpose | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) | | Ximalaya | Audio sharing | OLAP | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) | | Yandex Cloud | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | @@ -122,7 +133,9 @@ toc_title: Adopters | ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | +| Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | +| ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | Tesla | Electric vehicle and clean energy company | — | — | — | [Vacancy description, March 2021](https://news.ycombinator.com/item?id=26306170) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From ac44be2766b198c7653ab1ee477777f09e52797a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 31 Mar 2021 08:21:27 +0300 Subject: [PATCH 143/266] Update 01782_field_oom.sql --- tests/queries/0_stateless/01782_field_oom.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01782_field_oom.sql b/tests/queries/0_stateless/01782_field_oom.sql index 60326b09a7a..2609c589d94 100644 --- a/tests/queries/0_stateless/01782_field_oom.sql +++ b/tests/queries/0_stateless/01782_field_oom.sql @@ -1,2 +1,2 @@ -SET max_memory_usage = '1G'; +SET max_memory_usage = '500M'; SELECT sumMap([number], [number]) FROM system.numbers_mt; -- { serverError 241 } From 32cca8cfb6440b88d187023ecb31f508877af1cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 08:31:47 +0300 Subject: [PATCH 144/266] More --- docs/en/introduction/adopters.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 4d84895afa3..ebc90211594 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -12,8 +12,10 @@ toc_title: Adopters |---------|----------|---------|--------------|------------------------------------------------------------------------------|-----------| | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | | Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | +| AdScribe | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | +| Altinity | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | | ApiRoad | API marketplace | Analytics | — | — | [Blog post, Nov 2018, Mar 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | | Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | @@ -118,6 +120,7 @@ toc_title: Adopters | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | +| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | From 02021aaa3ed6a15070ada1c408b3201fb87427d0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 31 Mar 2021 08:49:33 +0300 Subject: [PATCH 145/266] Update datasketches.cmake --- cmake/find/datasketches.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/datasketches.cmake b/cmake/find/datasketches.cmake index 93ad9e7ed48..44ef324a9f2 100644 --- a/cmake/find/datasketches.cmake +++ b/cmake/find/datasketches.cmake @@ -26,4 +26,4 @@ endif() endif() -message (STATUS "Using datasketches=${USE_DATASKETCHES}: ${DATASKETCHES_INCLUDE_DIR} : ${DATASKETCHES_LIBRARY}") \ No newline at end of file +message (STATUS "Using datasketches=${USE_DATASKETCHES}: ${DATASKETCHES_INCLUDE_DIR} : ${DATASKETCHES_LIBRARY}") From 94d3ebb5eadce50719b1717d4889dc1669887a54 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 31 Mar 2021 09:19:35 +0300 Subject: [PATCH 146/266] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index ebc90211594..2c1a077d112 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -120,7 +120,7 @@ toc_title: Adopters | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | -| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | +| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | From 0bb44708c7da98a268a531a72eaca6603fe9f134 Mon Sep 17 00:00:00 2001 From: Slach Date: Wed, 31 Mar 2021 11:32:00 +0500 Subject: [PATCH 147/266] add description for parseDateTime64BestEffort, parseDateTime64BestEffortOrNull, parseDateTime64BestEffortOrZero sync EN and RU versions of `type-conversion-functions.md`, add notes about reinterpret and CAST functions --- .../functions/type-conversion-functions.md | 111 +++++++++- .../functions/type-conversion-functions.md | 198 +++++++++++++----- 2 files changed, 250 insertions(+), 59 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 8544356f895..d8d13d81d97 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -385,8 +385,6 @@ reinterpretAsUUID(fixed_string) - `fixed_string` — Big-endian byte string. [FixedString](../../sql-reference/data-types/fixedstring.md#fixedstring). -## reinterpret(x, T) {#type_conversion_function-reinterpret} - **Returned value** - The UUID type value. [UUID](../../sql-reference/data-types/uuid.md#uuid-data-type). @@ -398,9 +396,7 @@ String to UUID. Query: ``` sql -SELECT reinterpret(toInt8(-1), 'UInt8') as int_to_uint, - reinterpret(toInt8(1), 'Float32') as int_to_float, - reinterpret('1', 'UInt32') as string_to_int; +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); ``` Result: @@ -431,15 +427,51 @@ Result: └─────────────────────┘ ``` +## reinterpret(x, T) {#type_conversion_function-reinterpret} + +Use the same source in-memory bytes sequence for `x` value and reinterpret it to destination type + +Query: +```sql +SELECT reinterpret(toInt8(-1), 'UInt8') as int_to_uint, + reinterpret(toInt8(1), 'Float32') as int_to_float, + reinterpret('1', 'UInt32') as string_to_int; +``` + +Result: + +``` +┌─int_to_uint─┬─int_to_float─┬─string_to_int─┐ +│ 255 │ 1e-45 │ 49 │ +└─────────────┴──────────────┴───────────────┘ +``` + ## CAST(x, T) {#type_conversion_function-cast} -Converts input value `x` to the `T` data type. +Converts input value `x` to the `T` data type. Unlike to `reinterpret` function use external representation of `x` value. The syntax `CAST(x AS t)` is also supported. Note, that if value `x` does not fit the bounds of type T, the function overflows. For example, CAST(-1, 'UInt8') returns 255. -**Example** +**Examples** + +Query: + +```sql +SELECT + cast(toInt8(-1), 'UInt8') AS cast_int_to_uint, + cast(toInt8(1), 'Float32') AS cast_int_to_float, + cast('1', 'UInt32') AS cast_string_to_int +``` + +Result: + +``` +┌─cast_int_to_uint─┬─cast_int_to_float─┬─cast_string_to_int─┐ +│ 255 │ 1 │ 1 │ +└──────────────────┴───────────────────┴────────────────────┘ +``` Query: @@ -634,6 +666,7 @@ Result: ``` ## parseDateTimeBestEffort {#parsedatetimebesteffort} +## parseDateTime32BestEffort {#parsedatetime32besteffort} Converts a date and time in the [String](../../sql-reference/data-types/string.md) representation to [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) data type. @@ -822,10 +855,12 @@ Result: ``` ## parseDateTimeBestEffortOrNull {#parsedatetimebesteffortornull} +## parseDateTime32BestEffortOrNull {#parsedatetime32besteffortornull} -Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns null when it encounters a date format that cannot be processed. +Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns `NULL` when it encounters a date format that cannot be processed. ## parseDateTimeBestEffortOrZero {#parsedatetimebesteffortorzero} +## parseDateTime32BestEffortOrZero {#parsedatetime32besteffortorzero} Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed. @@ -1001,6 +1036,57 @@ Result: └─────────────────────────────────┘ ``` +## parseDateTime64BestEffort {#parsedatetime64besteffort} + +Same as [parseDateTimeBestEffort](#parsedatetimebesteffort) function but also parse milliseconds and microseconds and return `DateTime64(3)` or `DateTime64(6)` data types. + +**Syntax** + +``` sql +parseDateTime64BestEffort(time_string [, precision [, time_zone]]) +``` + +**Parameters** + +- `time_string` — String containing a date or date with time to convert. [String](../../sql-reference/data-types/string.md). +- `precision` — `3` for milliseconds, `6` for microseconds. Default `3`. Optional [UInt8](../../sql-reference/data-types/int-uint.md). +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Examples** + +Query: + +```sql +SELECT parseDateTime64BestEffort('2021-01-01') AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346') AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346',6) AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346',3,'Europe/Moscow') AS a, toTypeName(a) AS t +FORMAT PrettyCompactMonoBlcok +``` + +Result: + +``` +┌──────────────────────────a─┬─t──────────────────────────────┐ +│ 2021-01-01 01:01:00.123000 │ DateTime64(3) │ +│ 2021-01-01 00:00:00.000000 │ DateTime64(3) │ +│ 2021-01-01 01:01:00.123460 │ DateTime64(6) │ +│ 2020-12-31 22:01:00.123000 │ DateTime64(3, 'Europe/Moscow') │ +└────────────────────────────┴────────────────────────────────┘ +``` + +## parseDateTime64BestEffortOrNull {#parsedatetime32besteffortornull} + +Same as for [parseDateTime64BestEffort](#parsedatetime64besteffort) except that it returns `NULL` when it encounters a date format that cannot be processed. + +## parseDateTime64BestEffortOrZero {#parsedatetime64besteffortorzero} + +Same as for [parseDateTime64BestEffort](#parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed. + + ## toLowCardinality {#tolowcardinality} Converts input parameter to the [LowCardianlity](../../sql-reference/data-types/lowcardinality.md) version of same data type. @@ -1009,7 +1095,7 @@ To convert data from the `LowCardinality` data type use the [CAST](#type_convers **Syntax** -``` sql +```sql toLowCardinality(expr) ``` @@ -1027,7 +1113,7 @@ Type: `LowCardinality(expr_result_type)` Query: -``` sql +```sql SELECT toLowCardinality('1'); ``` @@ -1045,7 +1131,8 @@ Result: ## toUnixTimestamp64Nano {#tounixtimestamp64nano} -Converts a `DateTime64` to a `Int64` value with fixed sub-second precision. Input value is scaled up or down appropriately depending on it precision. Please note that output value is a timestamp in UTC, not in timezone of `DateTime64`. +Converts a `DateTime64` to a `Int64` value with fixed sub-second precision. +Input value is scaled up or down appropriately depending on it precision. Please note that output value is a timestamp in UTC, not in timezone of `DateTime64`. **Syntax** @@ -1078,6 +1165,8 @@ Result: └──────────────────────────────┘ ``` +Query: + ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 SELECT toUnixTimestamp64Nano(dt64); diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 1bb7396ae3a..fc1dd15f8e3 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -423,15 +423,51 @@ SELECT uuid = uuid2; └─────────────────────┘ ``` +## reinterpret(x, T) {#type_conversion_function-reinterpret} + +Использует туже самую исходную последовательность байт в памяти для значения `x` и переинтерпретирует ее как конечный тип данных + +Запрос: +```sql +SELECT reinterpret(toInt8(-1), 'UInt8') as int_to_uint, + reinterpret(toInt8(1), 'Float32') as int_to_float, + reinterpret('1', 'UInt32') as string_to_int; +``` + +Результат: + +``` +┌─int_to_uint─┬─int_to_float─┬─string_to_int─┐ +│ 255 │ 1e-45 │ 49 │ +└─────────────┴──────────────┴───────────────┘ +``` + ## CAST(x, T) {#type_conversion_function-cast} -Преобразует входное значение `x` в указанный тип данных `T`. +Преобразует входное значение `x` в указанный тип данных `T`. В отличии от функции `reinterpret` использует внешнее представление значения `x`. Поддерживается также синтаксис `CAST(x AS t)`. Обратите внимание, что если значение `x` не может быть преобразовано к типу `T`, возникает переполнение. Например, `CAST(-1, 'UInt8')` возвращает 255. -**Пример** +**Примеры** + +Запрос: + +```sql +SELECT + cast(toInt8(-1), 'UInt8') AS cast_int_to_uint, + cast(toInt8(1), 'Float32') AS cast_int_to_float, + cast('1', 'UInt32') AS cast_string_to_int +``` + +Результат: + +``` +┌─cast_int_to_uint─┬─cast_int_to_float─┬─cast_string_to_int─┐ +│ 255 │ 1 │ 1 │ +└──────────────────┴───────────────────┴────────────────────┘ +``` Запрос: @@ -511,7 +547,8 @@ SELECT cast(-1, 'UInt8') as uint8; ``` text ┌─uint8─┐ │ 255 │ -└───── +└───────┘ +``` Запрос: @@ -627,6 +664,7 @@ SELECT ``` ## parseDateTimeBestEffort {#parsedatetimebesteffort} +## parseDateTime32BestEffort {#parsedatetime32besteffort} Преобразует дату и время в [строковом](../../sql-reference/functions/type-conversion-functions.md) представлении к типу данных [DateTime](../../sql-reference/functions/type-conversion-functions.md#data_type-datetime). @@ -814,6 +852,16 @@ AS parseDateTimeBestEffortUS; └─────────────────────────——┘ ``` +## parseDateTimeBestEffortOrNull {#parsedatetimebesteffortornull} +## parseDateTime32BestEffortOrNull {#parsedatetime32besteffortornull} + +Работает также как [parseDateTimeBestEffort](#parsedatetimebesteffort), но возвращает `NULL` когда получает формат даты который не может быть обработан. + +## parseDateTimeBestEffortOrZero {#parsedatetimebesteffortorzero} +## parseDateTime32BestEffortOrZero {#parsedatetime32besteffortorzero} + +Работает также как [parseDateTimeBestEffort](#parsedatetimebesteffort), но возвращает нулевую дату или нулевую дату и время когда получает формат даты который не может быть обработан. + ## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} Работает аналогично функции [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но в отличие от нее возвращает `NULL`, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). @@ -986,9 +1034,100 @@ SELECT parseDateTimeBestEffortUSOrZero('02.2021') AS parseDateTimeBestEffortUSOr └─────────────────────────────────┘ ``` -## toUnixTimestamp64Milli -## toUnixTimestamp64Micro -## toUnixTimestamp64Nano +## parseDateTime64BestEffort {#parsedatetime64besteffort} + +Работает также как функция [parseDateTimeBestEffort](#parsedatetimebesteffort) но также понимамет милисекунды и микросекунды и возвращает `DateTime64(3)` или `DateTime64(6)` типы данных в зависимости от заданной точности. + +**Syntax** + +``` sql +parseDateTime64BestEffort(time_string [, precision [, time_zone]]) +``` + +**Parameters** + +- `time_string` — String containing a date or date with time to convert. [String](../../sql-reference/data-types/string.md). +- `precision` — `3` for milliseconds, `6` for microseconds. Default `3`. Optional [UInt8](../../sql-reference/data-types/int-uint.md). +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Примеры** + +Запрос: + +```sql +SELECT parseDateTime64BestEffort('2021-01-01') AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346') AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346',6) AS a, toTypeName(a) AS t +UNION ALL +SELECT parseDateTime64BestEffort('2021-01-01 01:01:00.12346',3,'Europe/Moscow') AS a, toTypeName(a) AS t +FORMAT PrettyCompactMonoBlcok +``` + +Результат: + +``` +┌──────────────────────────a─┬─t──────────────────────────────┐ +│ 2021-01-01 01:01:00.123000 │ DateTime64(3) │ +│ 2021-01-01 00:00:00.000000 │ DateTime64(3) │ +│ 2021-01-01 01:01:00.123460 │ DateTime64(6) │ +│ 2020-12-31 22:01:00.123000 │ DateTime64(3, 'Europe/Moscow') │ +└────────────────────────────┴────────────────────────────────┘ +``` + +## parseDateTime64BestEffortOrNull {#parsedatetime32besteffortornull} + +Работает также как функция [parseDateTime64BestEffort](#parsedatetime64besteffort) но возвращает `NULL` когда встречает формат даты который не может обработать. + +## parseDateTime64BestEffortOrZero {#parsedatetime64besteffortorzero} + +Работает также как функция [parseDateTime64BestEffort](#parsedatetimebesteffort) но возвращает "нулевую" дату и время когда встречает формат даты который не может обработать. + + +## toLowCardinality {#tolowcardinality} + +Преобразует входные данные в версию [LowCardianlity](../data-types/lowcardinality.md) того же типа данных. + +Чтобы преобразовать данные из типа `LowCardinality`, используйте функцию [CAST](#type_conversion_function-cast). Например, `CAST(x as String)`. + +**Синтаксис** + +```sql +toLowCardinality(expr) +``` + +**Аргументы** + +- `expr` — [выражение](../syntax.md#syntax-expressions), которое в результате преобразуется в один из [поддерживаемых типов данных](../data-types/index.md#data_types). + +**Возвращаемое значение** + +- Результат преобразования `expr`. + +Тип: `LowCardinality(expr_result_type)` + +**Пример** + +Запрос: + +```sql +SELECT toLowCardinality('1'); +``` + +Результат: + +```text +┌─toLowCardinality('1')─┐ +│ 1 │ +└───────────────────────┘ +``` + +## toUnixTimestamp64Milli {#tounixtimestamp64milli} + +## toUnixTimestamp64Micro {#tounixtimestamp64micro} + +## toUnixTimestamp64Nano {#tounixtimestamp64nano} Преобразует значение `DateTime64` в значение `Int64` с фиксированной точностью менее одной секунды. Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что возвращаемое значение - это временная метка в UTC, а не в часовом поясе `DateTime64`. @@ -1039,9 +1178,11 @@ SELECT toUnixTimestamp64Nano(dt64); └─────────────────────────────┘ ``` -## fromUnixTimestamp64Milli -## fromUnixTimestamp64Micro -## fromUnixTimestamp64Nano +## fromUnixTimestamp64Milli {#fromunixtimestamp64milli} + +## fromUnixTimestamp64Micro {#fromunixtimestamp64micro} + +## fromUnixTimestamp64Nano {#fromunixtimestamp64nano} Преобразует значение `Int64` в значение `DateTime64` с фиксированной точностью менее одной секунды и дополнительным часовым поясом. Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что входное значение обрабатывается как метка времени UTC, а не метка времени в заданном (или неявном) часовом поясе. @@ -1077,45 +1218,6 @@ SELECT fromUnixTimestamp64Milli(i64, 'UTC'); └──────────────────────────────────────┘ ``` -## toLowCardinality {#tolowcardinality} - -Преобразует входные данные в версию [LowCardianlity](../data-types/lowcardinality.md) того же типа данных. - -Чтобы преобразовать данные из типа `LowCardinality`, используйте функцию [CAST](#type_conversion_function-cast). Например, `CAST(x as String)`. - -**Синтаксис** - -```sql -toLowCardinality(expr) -``` - -**Аргументы** - -- `expr` — [выражение](../syntax.md#syntax-expressions), которое в результате преобразуется в один из [поддерживаемых типов данных](../data-types/index.md#data_types). - - -**Возвращаемое значение** - -- Результат преобразования `expr`. - -Тип: `LowCardinality(expr_result_type)` - -**Пример** - -Запрос: - -```sql -SELECT toLowCardinality('1'); -``` - -Результат: - -```text -┌─toLowCardinality('1')─┐ -│ 1 │ -└───────────────────────┘ -``` - ## formatRow {#formatrow} Преобразует произвольные выражения в строку заданного формата. From 9fd3cd2a212b02f38e8c48d0a9cd5b654ca80221 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Wed, 31 Mar 2021 08:34:52 +0200 Subject: [PATCH 148/266] Adding memory info to testflows RBAC --- tests/testflows/rbac/helper/common.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/testflows/rbac/helper/common.py b/tests/testflows/rbac/helper/common.py index c140e01f34f..a234a62eabd 100755 --- a/tests/testflows/rbac/helper/common.py +++ b/tests/testflows/rbac/helper/common.py @@ -28,6 +28,10 @@ def instrument_clickhouse_server_log(self, node=None, clickhouse_server_log="/va try: with And("adding test name start message to the clickhouse-server.log"): node.command(f"echo -e \"\\n-- start: {current().name} --\\n\" >> {clickhouse_server_log}") + with And("dump memory info"): + node.command(f"echo -e \"\\n-- {current().name} -- top --\\n\" && top -bn1") + node.command(f"echo -e \"\\n-- {current().name} -- df --\\n\" && df -h") + node.command(f"echo -e \"\\n-- {current().name} -- free --\\n\" && free -mh") yield finally: From f2a8e40012424411dcac8f91a8685086cca651e7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 31 Mar 2021 07:37:34 +0000 Subject: [PATCH 149/266] Do not rely on time --- tests/integration/test_storage_rabbitmq/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index bb8da289490..e489c2a134f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -253,6 +253,8 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): @pytest.mark.timeout(240) def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -260,6 +262,11 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_format = 'TSV', rabbitmq_queue_base = 'tsv', rabbitmq_row_delimiter = '\\n'; + CREATE TABLE test.view (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.rabbitmq; ''') credentials = pika.PlainCredentials('root', 'clickhouse') @@ -275,11 +282,10 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): channel.basic_publish(exchange='tsv', routing_key='', body=message) connection.close() - time.sleep(1) result = '' while True: - result += instance.query('SELECT * FROM test.rabbitmq ORDER BY key', ignore_error=True) + result = instance.query('SELECT * FROM test.view ORDER BY key') if rabbitmq_check_result(result): break From 7803d7303a4a79d0bcf9531f5d45b508e36e674e Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Wed, 31 Mar 2021 11:06:40 +0300 Subject: [PATCH 150/266] BOM deleted --- docs/ru/engines/table-engines/integrations/postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md index 919dbfe625b..0498a9e12a5 100644 --- a/docs/ru/engines/table-engines/integrations/postgresql.md +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -1,4 +1,4 @@ ---- +--- toc_priority: 11 toc_title: PostgreSQL --- From 0353775086ad39782c490983cd7d3b1e0b0629b1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 31 Mar 2021 12:28:25 +0300 Subject: [PATCH 151/266] Pass inner_columns to PredicateRewriteVisitorData::rewriteSubquery explicitly --- src/Interpreters/PredicateRewriteVisitor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index d4083e84639..6f28b9050df 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -42,7 +42,8 @@ void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_q void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &) { - is_rewrite |= rewriteSubquery(select_query, {}); + /// In this case inner_columns same as outer_columns from table_columns + is_rewrite |= rewriteSubquery(select_query, table_columns.columns.getNames()); } void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &) @@ -115,8 +116,7 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); if (outer_column_iterator != outer_columns.end()) { - const Names & column_names = inner_columns.empty() ? outer_columns : inner_columns; - identifier->setShortName(column_names[outer_column_iterator - outer_columns.begin()]); + identifier->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); } } From 9772d30754da88eb7ac46641c624f0939b2e3954 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 Mar 2021 13:21:30 +0300 Subject: [PATCH 152/266] Fixed performance tests --- src/Dictionaries/CacheDictionary.cpp | 25 +- src/Dictionaries/DictionaryHelpers.h | 102 ++++++--- src/Dictionaries/DirectDictionary.cpp | 43 ++-- src/Dictionaries/HashedDictionary.cpp | 318 ++++++++++++-------------- src/Dictionaries/HashedDictionary.h | 26 +++ 5 files changed, 279 insertions(+), 235 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index f1abe98b454..85f6cb98e8c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -142,8 +142,8 @@ Columns CacheDictionary::getColumns( dict_struct.validateKeyTypes(key_types); Arena complex_keys_arena; - DictionaryKeysExtractor extractor(key_columns, complex_keys_arena); - auto & keys = extractor.getKeys(); + DictionaryKeysExtractor extractor(key_columns, &complex_keys_arena); + auto keys = extractor.extractAllKeys(); return getColumnsImpl(attribute_names, key_columns, keys, default_values_columns); } @@ -282,8 +282,8 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k dict_struct.validateKeyTypes(key_types); Arena complex_keys_arena; - DictionaryKeysExtractor extractor(key_columns, complex_keys_arena); - const auto & keys = extractor.getKeys(); + DictionaryKeysExtractor extractor(key_columns, &complex_keys_arena); + const auto keys = extractor.extractAllKeys(); /// We make empty request just to fetch if keys exists DictionaryStorageFetchRequest request(dict_struct, {}, {}); @@ -531,8 +531,8 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr requested_keys_extractor(update_unit_ptr->key_columns, update_unit_ptr->complex_key_arena); - const auto & requested_keys = requested_keys_extractor.getKeys(); + DictionaryKeysExtractor requested_keys_extractor(update_unit_ptr->key_columns, &update_unit_ptr->complex_key_arena); + const auto requested_keys = requested_keys_extractor.extractAllKeys(); HashSet not_found_keys; @@ -598,23 +598,26 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr keys_extractor(key_columns, update_unit_ptr->complex_key_arena); - const auto & keys_extracted_from_block = keys_extractor.getKeys(); + DictionaryKeysExtractor keys_extractor(key_columns, &update_unit_ptr->complex_key_arena); + const size_t keys_extracted_from_block_size = keys_extractor.getKeysSize(); for (size_t index_of_attribute = 0; index_of_attribute < fetched_columns_during_update.size(); ++index_of_attribute) { auto & column_to_update = fetched_columns_during_update[index_of_attribute]; auto column = block.safeGetByPosition(skip_keys_size_offset + index_of_attribute).column; - column_to_update->assumeMutable()->insertRangeFrom(*column, 0, keys_extracted_from_block.size()); + column_to_update->assumeMutable()->insertRangeFrom(*column, 0, keys_extracted_from_block_size); } - for (size_t i = 0; i < keys_extracted_from_block.size(); ++i) + for (size_t i = 0; i < keys_extracted_from_block_size; ++i) { - auto fetched_key_from_source = keys_extracted_from_block[i]; + auto fetched_key_from_source = keys_extractor.extractCurrentKey(); + not_found_keys.erase(fetched_key_from_source); update_unit_ptr->requested_keys_to_fetched_columns_during_update_index[fetched_key_from_source] = found_keys_size; found_keys_in_source.emplace_back(fetched_key_from_source); ++found_keys_size; + + keys_extractor.rollbackCurrentKey(); } } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 21ebe3e4199..0be1ebb7728 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -302,62 +302,96 @@ public: using KeyType = std::conditional_t; static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor"); - explicit DictionaryKeysExtractor(const Columns & key_columns, Arena & existing_arena) + explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_) + : key_columns(key_columns_) + , complex_key_arena(complex_key_arena_) { assert(!key_columns.empty()); if constexpr (key_type == DictionaryKeyType::simple) { - simple_key_column = key_columns.front()->convertToFullColumnIfConst(); - - const auto * vector_col = checkAndGetColumn>(simple_key_column.get()); + key_columns[0] = key_columns[0]->convertToFullColumnIfConst(); + const auto * vector_col = checkAndGetColumn>(key_columns[0].get()); if (!vector_col) - throw Exception{ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"}; + throw Exception(ErrorCodes::TYPE_MISMATCH, "Column type mismatch for simple key expected UInt64"); + } + + keys_size = key_columns.front()->size(); + } + + inline size_t getKeysSize() const + { + return keys_size; + } + + inline size_t getCurrentKeyIndex() const + { + return current_key_index; + } + + inline KeyType extractCurrentKey() + { + assert(current_key_index < keys_size); + + if constexpr (key_type == DictionaryKeyType::simple) + { + const auto & column_vector = static_cast &>(*key_columns[0]); + const auto & data = column_vector.getData(); + + auto key = data[current_key_index]; + ++current_key_index; + return key; } else - complex_keys_serialized = deserializeKeyColumnsInArena(key_columns, existing_arena); - } - - const PaddedPODArray & getKeys() const - { - if constexpr (key_type == DictionaryKeyType::simple) - return static_cast *>(simple_key_column.get())->getData(); - else - return complex_keys_serialized; - } - -private: - static PaddedPODArray deserializeKeyColumnsInArena(const Columns & key_columns, Arena & temporary_arena) - { - size_t keys_size = key_columns.front()->size(); - - PaddedPODArray result; - result.reserve(keys_size); - - PaddedPODArray temporary_column_data(key_columns.size()); - - for (size_t key_index = 0; key_index < keys_size; ++key_index) { size_t allocated_size_for_columns = 0; const char * block_start = nullptr; - for (size_t column_index = 0; column_index < key_columns.size(); ++column_index) + for (const auto & column : key_columns) { - const auto & column = key_columns[column_index]; - temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_arena, block_start); - allocated_size_for_columns += temporary_column_data[column_index].size; + StringRef serialized_data = column->serializeValueIntoArena(current_key_index, *complex_key_arena, block_start); + allocated_size_for_columns += serialized_data.size; } - result.push_back(StringRef{block_start, allocated_size_for_columns}); + ++current_key_index; + current_complex_key = StringRef{block_start, allocated_size_for_columns}; + return current_complex_key; + } + } + + void rollbackCurrentKey() const + { + if constexpr (key_type == DictionaryKeyType::complex) + complex_key_arena->rollback(current_complex_key.size); + } + + PaddedPODArray extractAllKeys() + { + PaddedPODArray result; + result.reserve(keys_size - current_key_index); + + for (; current_key_index < keys_size;) + { + auto value = extractCurrentKey(); + result.emplace_back(value); } return result; } - PaddedPODArray complex_keys_serialized; - ColumnPtr simple_key_column; + void reset() + { + current_key_index = 0; + } +private: + Columns key_columns; + size_t keys_size = 0; + size_t current_key_index = 0; + + KeyType current_complex_key {}; + Arena * complex_key_arena; }; /** diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 4ee6d3c346d..87af00e56a6 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -41,14 +41,18 @@ ColumnPtr DirectDictionary::getColumn( if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena complex_key_arena; + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; + + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; + + DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + const auto requested_keys = extractor.extractAllKeys(); const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type); DefaultValueProvider default_value_provider(attribute.null_value, default_values_column); - DictionaryKeysExtractor extractor(key_columns, complex_key_arena); - const auto & requested_keys = extractor.getKeys(); - HashMap key_to_fetched_index; key_to_fetched_index.reserve(requested_keys.size()); @@ -75,19 +79,20 @@ ColumnPtr DirectDictionary::getColumn( block_columns.erase(block_columns.begin()); } - DictionaryKeysExtractor block_keys_extractor(block_key_columns, complex_key_arena); - const auto & block_keys = block_keys_extractor.getKeys(); - size_t block_keys_size = block_keys.size(); + DictionaryKeysExtractor block_keys_extractor(block_key_columns, temporary_complex_key_arena); + size_t block_keys_size = block_keys_extractor.getKeysSize(); const auto & block_column = block.safeGetByPosition(dictionary_keys_size + requested_attribute_index).column; fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys_size); for (size_t block_key_index = 0; block_key_index < block_keys_size; ++block_key_index) { - const auto & block_key = block_keys[block_key_index]; + const auto block_key = block_keys_extractor.extractCurrentKey(); key_to_fetched_index[block_key] = fetched_key_index; ++fetched_key_index; + + block_keys_extractor.rollbackCurrentKey(); } block_key_columns.clear(); @@ -126,11 +131,15 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena complex_key_arena; + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; - DictionaryKeysExtractor requested_keys_extractor(key_columns, complex_key_arena); - const auto & requested_keys = requested_keys_extractor.getKeys(); - size_t requested_keys_size = requested_keys.size(); + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; + + DictionaryKeysExtractor requested_keys_extractor(key_columns, temporary_complex_key_arena); + const auto requested_keys = requested_keys_extractor.extractAllKeys(); + size_t requested_keys_size = requested_keys_extractor.getKeysSize(); HashMap requested_key_to_index; requested_key_to_index.reserve(requested_keys_size); @@ -163,16 +172,20 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & block_columns.erase(block_columns.begin()); } - DictionaryKeysExtractor block_keys_extractor(block_key_columns, complex_key_arena); - const auto & block_keys = block_keys_extractor.getKeys(); + DictionaryKeysExtractor block_keys_extractor(block_key_columns, temporary_complex_key_arena); + size_t block_keys_size = block_keys_extractor.getKeysSize(); - for (const auto & block_key : block_keys) + for (size_t i = 0; i < block_keys_size; ++i) { + auto block_key = block_keys_extractor.extractCurrentKey(); + const auto * it = requested_key_to_index.find(block_key); assert(it); size_t result_data_found_index = it->getMapped(); result_data[result_data_found_index] = true; + + block_keys_extractor.rollbackCurrentKey(); } block_key_columns.clear(); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 6bd1d58ca99..37520e5a2ec 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -67,186 +67,78 @@ ColumnPtr HashedDictionary::getColumn( const DataTypes & key_types [[maybe_unused]], const ColumnPtr & default_values_column) const { - // if constexpr (dictionary_key_type == DictionaryKeyType::simple) - // { - // ColumnPtr result; - - // PaddedPODArray backup_storage; - // const auto & ids = getColumnVectorData(this, key_columns.front(), backup_storage); - - // auto size = ids.size(); - - // const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); - // DefaultValueProvider default_value_provider(dictionary_attribute.null_value, default_values_column); - - // size_t index = dict_struct.attribute_name_to_index.find(attribute_name)->second; - // const auto & attribute = attributes[index]; - - // auto type_call = [&](const auto & dictionary_attribute_type) - // { - // using Type = std::decay_t; - // using AttributeType = typename Type::AttributeType; - // using ValueType = DictionaryValueType; - // using ColumnProvider = DictionaryAttributeColumnProvider; - - // auto & container = std::get>(attribute.container); - - // auto column = ColumnProvider::getColumn(dictionary_attribute, size); - - // if constexpr (std::is_same_v) - // { - // auto * out = column.get(); - - // const auto rows = ext::size(ids); - - // for (const auto i : ext::range(0, rows)) - // { - // const auto it = container.find(ids[i]); - - // if (it != container.end()) - // { - // StringRef item = getValueFromCell(it); - // out->insertData(item.data, item.size); - // } - // else - // { - // Field default_value = default_value_provider.getDefaultValue(i); - // String & default_value_string = default_value.get(); - // out->insertData(default_value_string.data(), default_value_string.size()); - // } - // } - // } - // else - // { - // auto & out = column->getData(); - // const auto rows = ext::size(ids); - - // for (const auto i : ext::range(0, rows)) - // { - // const auto it = container.find(ids[i]); - - // if (it != container.end()) - // { - // auto item = getValueFromCell(it); - // out[i] = item; - // } - // else - // { - // Field default_value = default_value_provider.getDefaultValue(i); - // out[i] = default_value.get>(); - // } - // } - - // } - - // result = std::move(column); - // }; - - // callOnDictionaryAttributeType(attribute.type, type_call); - // query_count.fetch_add(ids.size(), std::memory_order_relaxed); - - // return result; - // } - // else - // return nullptr; - if constexpr (dictionary_key_type == DictionaryKeyType::complex) + if (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena temporary_complex_key_arena; + ColumnPtr result; - const DictionaryAttribute & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); - DefaultValueProvider default_value_provider(dictionary_attribute.null_value, default_values_column); + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; + + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - /// TODO: Check extractor performance DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); - const auto & requested_keys = extractor.getKeys(); - auto result_column = dictionary_attribute.nested_type->createColumn(); + const size_t size = extractor.getKeysSize(); - size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; - const auto & attribute = attributes[attribute_index]; + 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]; - size_t requested_keys_size = requested_keys.size(); + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (attribute.is_nullable_set) + { + col_null_map_to = ColumnUInt8::create(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 ColumnType = std::conditional_t< - std::is_same_v, - ColumnString, - std::conditional_t, ColumnDecimal, ColumnVector>>; + using ColumnProvider = DictionaryAttributeColumnProvider; - const auto & attribute_container = std::get>(attribute.container); - ColumnType & result_column_typed = static_cast(*result_column); + const auto attribute_null_value = std::get(attribute.null_values); + AttributeType null_value = static_cast(attribute_null_value); + DictionaryDefaultValueExtractor default_value_extractor(std::move(null_value), default_values_column); - if constexpr (std::is_same_v) + auto column = ColumnProvider::getColumn(dictionary_attribute, size); + + if constexpr (std::is_same_v) { - result_column_typed.reserve(requested_keys_size); + auto * out = column.get(); - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) - { - auto & requested_key = requested_keys[requested_key_index]; - auto it = attribute_container.find(requested_key); - - if (it != attribute_container.end()) - { - StringRef item = getValueFromCell(it); - result_column->insertData(item.data, item.size); - } - else - { - Field default_value = default_value_provider.getDefaultValue(requested_key_index); - String & default_value_string = default_value.get(); - result_column->insertData(default_value_string.data(), default_value_string.size()); - } - } + getItemsImpl( + attribute, + extractor, + [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, + [&](const size_t row) { (*vec_null_map_to)[row] = true; }, + default_value_extractor); } else { - auto & result_data = result_column_typed.getData(); - result_data.resize_fill(requested_keys_size); + auto & out = column->getData(); - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) - { - auto & requested_key = requested_keys[requested_key_index]; - - auto it = attribute_container.find(requested_key); - - if (it != attribute_container.end()) - { - ValueType item = getValueFromCell(it); - result_data[requested_key_index] = item; - } - else - { - auto default_value_to_insert = default_value_provider.getDefaultValue(requested_key_index); - result_data[requested_key_index] = default_value_to_insert.get>(); - } - } + getItemsImpl( + attribute, + extractor, + [&](const size_t row, const auto value) { return out[row] = value; }, + [&](const size_t row) { (*vec_null_map_to)[row] = true; }, + default_value_extractor); } + + result = std::move(column); }; callOnDictionaryAttributeType(attribute.type, type_call); if (attribute.is_nullable_set) - { - ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(requested_keys_size, false); - ColumnUInt8::Container& vec_null_map_to = col_null_map_to->getData(); + result = ColumnNullable::create(result, std::move(col_null_map_to)); - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) - { - auto key = requested_keys[requested_key_index]; - vec_null_map_to[requested_key_index] = (attribute.is_nullable_set->find(key) != nullptr); - } - - result_column = ColumnNullable::create(std::move(result_column), std::move(col_null_map_to)); - } - - query_count.fetch_add(requested_keys.size(), std::memory_order_relaxed); - - return result_column; + return result; } template @@ -255,15 +147,18 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co if (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - /// TODO: Check performance of extractor - Arena complex_keys_arena; - DictionaryKeysExtractor extractor(key_columns, complex_keys_arena); + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; - const auto & keys = extractor.getKeys(); - size_t keys_size = keys.size(); + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; + + DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + + size_t keys_size = extractor.getKeysSize(); auto result = ColumnUInt8::create(keys_size, false); - auto& out = result->getData(); + auto & out = result->getData(); if (attributes.empty()) { @@ -272,17 +167,20 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co } const auto & attribute = attributes.front(); + bool is_attribute_nullable = attribute.is_nullable_set.has_value(); getAttributeContainer(0, [&](const auto & container) { for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) { - const auto & requested_key = keys[requested_key_index]; + auto requested_key = extractor.extractCurrentKey(); out[requested_key_index] = container.find(requested_key) != container.end(); - if (unlikely(attribute.is_nullable_set) && !out[requested_key_index]) + if (is_attribute_nullable && !out[requested_key_index]) out[requested_key_index] = attribute.is_nullable_set->find(requested_key) != nullptr; + + extractor.rollbackCurrentKey(); } }); @@ -425,7 +323,22 @@ void HashedDictionary::createAttributes() auto is_nullable_set = dictionary_attribute.is_nullable ? std::make_optional() : std::optional{}; std::unique_ptr string_arena = std::is_same_v ? std::make_unique() : nullptr; - Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), CollectionType(), std::move(string_arena)}; + ValueType default_value; + + if constexpr (std::is_same_v) + { + string_arena = std::make_unique(); + + const auto & string_null_value = dictionary_attribute.null_value.get(); + const size_t string_null_value_size = string_null_value.size(); + + const char * string_in_arena = string_arena->insert(string_null_value.data(), string_null_value_size); + default_value = {string_in_arena, string_null_value_size}; + } + else + default_value = dictionary_attribute.null_value.get>(); + + Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), default_value, CollectionType(), std::move(string_arena)}; attributes.emplace_back(std::move(attribute)); }; @@ -446,6 +359,7 @@ void HashedDictionary::updateData() /// We are using this to keep saved data if input stream consists of multiple blocks if (!saved_block) saved_block = std::make_shared(block.cloneEmpty()); + for (const auto attribute_idx : ext::range(0, attributes.size() + 1)) { const IColumn & update_column = *block.getByPosition(attribute_idx).column.get(); @@ -457,8 +371,6 @@ void HashedDictionary::updateData() } else { - Arena temporary_complex_key_arena; - size_t skip_keys_size_offset = dict_struct.getKeysSize(); Columns saved_block_key_columns; @@ -468,8 +380,15 @@ void HashedDictionary::updateData() for (size_t i = 0; i < skip_keys_size_offset; ++i) saved_block_key_columns.emplace_back(saved_block->safeGetByPosition(i).column); + + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; + + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; + DictionaryKeysExtractor saved_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); - const auto & saved_keys_extracted_from_block = saved_keys_extractor.getKeys(); + const auto & saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys(); auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); @@ -485,13 +404,14 @@ void HashedDictionary::updateData() block_key_columns.emplace_back(block.safeGetByPosition(i).column); DictionaryKeysExtractor block_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); - const auto & keys_extracted_from_block = block_keys_extractor.getKeys(); + const auto keys_extracted_from_block_size = block_keys_extractor.getKeysSize(); absl::flat_hash_map, DefaultHash> update_keys; - for (size_t row = 0; row < keys_extracted_from_block.size(); ++row) + for (size_t row = 0; row < keys_extracted_from_block_size; ++row) { - const auto key = keys_extracted_from_block[row]; + const auto key = block_keys_extractor.extractCurrentKey(); update_keys[key].push_back(row); + block_keys_extractor.rollbackCurrentKey(); } IColumn::Filter filter(saved_keys_extracted_from_block.size()); @@ -527,7 +447,11 @@ void HashedDictionary::updateData() template void HashedDictionary::blockToAttributes(const Block & block [[maybe_unused]]) { - Arena temporary_complex_key_arena; + Arena temporary_complex_keys_arena_holder [[maybe_unused]]; + Arena * temporary_complex_key_arena = nullptr; + + if constexpr (dictionary_key_type == DictionaryKeyType::complex) + temporary_complex_key_arena = &temporary_complex_keys_arena_holder; size_t skip_keys_size_offset = dict_struct.getKeysSize(); @@ -539,7 +463,7 @@ void HashedDictionary::blockToAttributes(const Bloc key_columns.emplace_back(block.safeGetByPosition(i).column); DictionaryKeysExtractor keys_extractor(key_columns, temporary_complex_key_arena); - const auto & keys_extracted_from_block = keys_extractor.getKeys(); + const size_t keys_size = keys_extractor.getKeysSize(); Field column_value_to_insert; @@ -547,22 +471,25 @@ void HashedDictionary::blockToAttributes(const Bloc { const IColumn & attribute_column = *block.safeGetByPosition(skip_keys_size_offset + attribute_index).column; auto & attribute = attributes[attribute_index]; + bool attribute_is_nullable = attribute.is_nullable_set.has_value(); getAttributeContainer(attribute_index, [&](auto & container) { using ContainerType = std::decay_t; using AttributeValueType = typename ContainerType::mapped_type; - for (size_t key_index = 0; key_index < keys_extracted_from_block.size(); ++key_index) + for (size_t key_index = 0; key_index < keys_size; ++key_index) { - auto key = keys_extracted_from_block[key_index]; + auto key = keys_extractor.extractCurrentKey(); + auto it = container.find(key); + bool key_is_nullable_and_already_exists = attribute_is_nullable && attribute.is_nullable_set->find(key) != nullptr; - if (attribute.is_nullable_set && (attribute.is_nullable_set->find(key) != nullptr)) - continue; - - if (it != container.end()) + if (key_is_nullable_and_already_exists || it != container.end()) + { + keys_extractor.rollbackCurrentKey(); continue; + } if constexpr (std::is_same_v) key = copyKeyInArena(key); @@ -572,6 +499,7 @@ void HashedDictionary::blockToAttributes(const Bloc if (attribute.is_nullable_set && column_value_to_insert.isNull()) { attribute.is_nullable_set->insert(key); + keys_extractor.rollbackCurrentKey(); continue; } @@ -592,7 +520,11 @@ void HashedDictionary::blockToAttributes(const Bloc } ++element_count; + + keys_extractor.rollbackCurrentKey(); } + + keys_extractor.reset(); }); } } @@ -617,6 +549,42 @@ void HashedDictionary::resize(size_t added_rows) } } +template +template +void HashedDictionary::getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value [[maybe_unused]], + NullableValueSetter && set_nullable_value [[maybe_unused]], + DefaultValueExtractor & default_value_extractor) const +{ + const auto & attribute_container = std::get>(attribute.container); + const size_t keys_size = keys_extractor.getKeysSize(); + + // bool is_attribute_nullable = attribute.is_nullable_set.has_value(); + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + const auto it = attribute_container.find(key); + + if (it != attribute_container.end()) + set_value(key_index, getValueFromCell(it)); + else + { + // if (is_attribute_nullable && attribute.is_nullable_set->find(key) != nullptr) + // set_nullable_value(key_index); + // else + set_value(key_index, default_value_extractor[key_index]); + } + + keys_extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); +} + template StringRef HashedDictionary::copyKeyInArena(StringRef key) { @@ -806,7 +774,7 @@ void registerDictionaryHashed(DictionaryFactory & factory) factory.registerLayout("sparse_hashed", [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false); factory.registerLayout("complex_key_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true); factory.registerLayout("complex_key_sparse_hashed", [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true); diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index c7cb59c2e3f..3882b669324 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -139,6 +139,24 @@ private: AttributeUnderlyingType type; std::optional is_nullable_set; + std::variant< + UInt8, + UInt16, + UInt32, + UInt64, + UInt128, + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + StringRef> + null_values; + std::variant< CollectionType, CollectionType, @@ -171,6 +189,14 @@ private: void calculateBytesAllocated(); + template + void getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value, + NullableValueSetter && set_nullable_value, + DefaultValueExtractor & default_value_extractor) const; + template void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); From 061e3c7d81c75788355f94ec9bf3492d73206dab Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 31 Mar 2021 18:28:58 +0800 Subject: [PATCH 153/266] Correctly place debug helpers --- cmake/warnings.cmake | 5 ----- src/CMakeLists.txt | 5 +++++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 8122e9ef31e..a398c59e981 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -11,11 +11,6 @@ if (NOT MSVC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra") endif () -if (USE_DEBUG_HELPERS) - set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") -endif () - # Add some warnings that are not available even with -Wall -Wextra -Wpedantic. # Intended for exploration of new compiler warnings that may be found useful. # Applies to clang only diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 805941f7dcc..7a3d2861da7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -27,6 +27,11 @@ configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) configure_file (Core/config_core.h.in ${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h) +if (USE_DEBUG_HELPERS) + set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") +endif () + if (COMPILER_GCC) # If we leave this optimization enabled, gcc-7 replaces a pair of SSE intrinsics (16 byte load, store) with a call to memcpy. # It leads to slow code. This is compiler bug. It looks like this: From 5cd1dc485fbc5b8767f76e3abbb81c63218a6532 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 31 Mar 2021 14:14:46 +0100 Subject: [PATCH 154/266] Delete temporary fetch directory if it already exists It could be left there after a network timeout, a system crash, or checksum issues. The downside of this PR is that it would be harder to debug checksum issues if replication will continuously delete and try to fetch the same part. --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index cf8de4456dd..b9ff7b41af6 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -547,7 +547,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; if (disk->exists(part_download_path)) - throw Exception("Directory " + fullPath(disk, part_download_path) + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + { + LOG_WARNING(log, "Directory {} already exists, probably result of a failed fetch. Will remove it before fetching part.", + fullPath(disk, part_download_path)); + disk->removeRecursive(part_download_path); + } disk->createDirectories(part_download_path); From eb4d6cfd72ba625ada5aa20a29ab5d743d791705 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 31 Mar 2021 16:26:57 +0300 Subject: [PATCH 155/266] more stable tests --- .../queries/0_stateless/00976_system_stop_ttl_merges.sql | 4 ++-- .../0_stateless/01666_merge_tree_max_query_limit.sh | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql b/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql index 41f2428d9e6..b27e4275d5d 100644 --- a/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql +++ b/tests/queries/0_stateless/00976_system_stop_ttl_merges.sql @@ -2,7 +2,7 @@ drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day; -system stop ttl merges; +system stop ttl merges ttl; insert into ttl values (toDateTime('2000-10-10 00:00:00'), 1), (toDateTime('2000-10-10 00:00:00'), 2) insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3), (toDateTime('2100-10-10 00:00:00'), 4); @@ -11,7 +11,7 @@ select sleep(1) format Null; -- wait if very fast merge happen optimize table ttl partition 10 final; select * from ttl order by d, a; -system start ttl merges; +system start ttl merges ttl; optimize table ttl partition 10 final; select * from ttl order by d, a; diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index e32a83c9560..ce6aa1dee5d 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -18,9 +18,11 @@ settings index_granularity = 1, max_concurrent_queries = 1, min_marks_to_honor_m insert into simple select number, number + 100 from numbers(1000); " +query_id = "long_running_query-$CLICKHOUSE_DATABASE" + echo "Spin up a long running query" -${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" > /dev/null 2>&1 & -wait_for_query_to_start 'long_running_query' +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "$query_id" > /dev/null 2>&1 & +wait_for_query_to_start "$query_id" # query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled echo "Check if another query with some marks to read is throttled" @@ -61,7 +63,7 @@ CODE=$? [ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; echo "yes" -${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = 'long_running_query' SYNC" +${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '$query_id' SYNC" wait ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " From ca5158a9c23ebd8f4864fee93febe79923cb63e9 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 31 Mar 2021 17:02:55 +0300 Subject: [PATCH 156/266] fix --- tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index ce6aa1dee5d..11a68cf6607 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -18,7 +18,7 @@ settings index_granularity = 1, max_concurrent_queries = 1, min_marks_to_honor_m insert into simple select number, number + 100 from numbers(1000); " -query_id = "long_running_query-$CLICKHOUSE_DATABASE" +query_id="long_running_query-$CLICKHOUSE_DATABASE" echo "Spin up a long running query" ${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "$query_id" > /dev/null 2>&1 & From 0e75f7bad5fb6b387982c1035149988389a4b25f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 31 Mar 2021 17:04:57 +0300 Subject: [PATCH 157/266] fix error message for invalid window frame start --- src/Parsers/ExpressionElementParsers.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 913813d5486..a15dbf3ea25 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -610,8 +610,8 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p node->frame.begin_preceding = false; if (node->frame.begin_type == WindowFrame::BoundaryType::Unbounded) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Frame start UNBOUNDED FOLLOWING is not implemented"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame start cannot be UNBOUNDED FOLLOWING"); } } else @@ -667,8 +667,8 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p node->frame.end_preceding = true; if (node->frame.end_type == WindowFrame::BoundaryType::Unbounded) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Frame end UNBOUNDED PRECEDING is not implemented"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame end cannot be UNBOUNDED PRECEDING"); } } else if (keyword_following.ignore(pos, expected)) From f2e6a0a97d6b1e73c07ab7c9abaf25c56d579a0e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 Mar 2021 17:46:13 +0300 Subject: [PATCH 158/266] Added function dictGetOrNull --- .../FunctionsExternalDictionaries.cpp | 1 + src/Functions/FunctionsExternalDictionaries.h | 146 +++++++++++++++++- .../01780_dict_get_or_null.reference | 18 +++ .../0_stateless/01780_dict_get_or_null.sql | 116 ++++++++++++++ 4 files changed, 279 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01780_dict_get_or_null.reference create mode 100644 tests/queries/0_stateless/01780_dict_get_or_null.sql diff --git a/src/Functions/FunctionsExternalDictionaries.cpp b/src/Functions/FunctionsExternalDictionaries.cpp index f037a3bd808..6bb5c476b1a 100644 --- a/src/Functions/FunctionsExternalDictionaries.cpp +++ b/src/Functions/FunctionsExternalDictionaries.cpp @@ -40,6 +40,7 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction(); } } diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 2c322698327..92622b78cab 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -19,6 +19,7 @@ #include #include #include +#include #include @@ -148,7 +149,6 @@ public: String getName() const override { return name; } -private: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } @@ -238,6 +238,7 @@ private: return dictionary->hasKeys({key_column, range_col}, {std::make_shared(), range_col_type}); } +private: mutable FunctionDictHelper helper; }; @@ -302,7 +303,7 @@ public: } if (types.size() > 1) - return std::make_shared(types); + return std::make_shared(types, attribute_names); else return types.front(); } @@ -701,6 +702,147 @@ using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault, NameDictGetDecimal128OrDefault>; using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault; +class FunctionDictGetOrNull final : public IFunction +{ +public: + static constexpr auto name = "dictGetOrNull"; + + static FunctionPtr create(const Context &context) + { + return std::make_shared(context); + } + + explicit FunctionDictGetOrNull(const Context & context_) + : dictionary_get_func_impl(context_) + , dictionary_has_func_impl(context_) + {} + + String getName() const override { return name; } + +private: + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + bool useDefaultImplementationForNulls() const override { return false; } + + bool isDeterministic() const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + + bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override + { + return dictionary_get_func_impl.isInjective(sample_columns); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments); + + WhichDataType result_data_type(result_type); + if (result_data_type.isTuple()) + { + const auto & data_type_tuple = static_cast(*result_type); + auto elements_types_copy = data_type_tuple.getElements(); + for (auto & element_type : elements_types_copy) + element_type = makeNullable(element_type); + + result_type = std::make_shared(elements_types_copy, data_type_tuple.getElementNames()); + } + else + result_type = makeNullable(result_type); + + return result_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + auto dictionary_name_argument_column = arguments[0]; + auto keys_argument_column = arguments[2]; + + ColumnsWithTypeAndName dict_has_arguments; + dict_has_arguments.reserve(arguments.size() - 1); + + for (size_t i = 0; i < arguments.size(); ++i) + { + if (i == 1) + continue; + + dict_has_arguments.emplace_back(arguments[i]); + } + + auto is_key_in_dictionary_column = dictionary_has_func_impl.executeImpl(dict_has_arguments, std::make_shared(), input_rows_count); + auto is_key_in_dictionary_column_mutable = is_key_in_dictionary_column->assumeMutable(); + ColumnVector & is_key_in_dictionary_column_typed = assert_cast &>(*is_key_in_dictionary_column_mutable); + auto & is_key_in_dictionary_data = is_key_in_dictionary_column_typed.getData(); + + auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments); + auto dictionary_get_result_column = dictionary_get_func_impl.executeImpl(arguments, result_type, input_rows_count); + + for (auto & key : is_key_in_dictionary_data) + key = !key; + + ColumnPtr result; + + WhichDataType result_data_type(result_type); + auto dictionary_get_result_column_mutable = dictionary_get_result_column->assumeMutable(); + + if (result_data_type.isTuple()) + { + ColumnTuple & column_tuple = assert_cast(*dictionary_get_result_column_mutable); + + const auto & columns = column_tuple.getColumns(); + size_t tuple_size = columns.size(); + + MutableColumns new_columns(tuple_size); + for (size_t tuple_column_index = 0; tuple_column_index < tuple_size; ++tuple_column_index) + { + auto nullable_column_map = ColumnVector::create(); + auto & nullable_column_map_data = nullable_column_map->getData(); + nullable_column_map_data.assign(is_key_in_dictionary_data); + + auto mutable_column = columns[tuple_column_index]->assumeMutable(); + if (ColumnNullable * nullable_column = typeid_cast(mutable_column.get())) + { + auto & null_map_data = nullable_column->getNullMapData(); + addNullMap(null_map_data, is_key_in_dictionary_data); + new_columns[tuple_column_index] = std::move(mutable_column); + } + else + new_columns[tuple_column_index] = ColumnNullable::create(std::move(mutable_column), std::move(nullable_column_map)); + } + + result = ColumnTuple::create(std::move(new_columns)); + } + else + { + if (ColumnNullable * nullable_column = typeid_cast(dictionary_get_result_column_mutable.get())) + { + auto & null_map_data = nullable_column->getNullMapData(); + addNullMap(null_map_data, is_key_in_dictionary_data); + result = std::move(dictionary_get_result_column); + } + else + result = ColumnNullable::create(std::move(dictionary_get_result_column), std::move(is_key_in_dictionary_column_mutable)); + } + + return result; + } + + static void addNullMap(PaddedPODArray & null_map, PaddedPODArray & null_map_to_add) + { + assert(null_map.size() == null_map_to_add.size()); + + for (size_t i = 0; i < null_map.size(); ++i) + null_map[i] = null_map[i] || null_map_to_add[i]; + } + + const FunctionDictGetNoType dictionary_get_func_impl; + const FunctionDictHas dictionary_has_func_impl; +}; /// Functions to work with hierarchies. class FunctionDictGetHierarchy final : public IFunction diff --git a/tests/queries/0_stateless/01780_dict_get_or_null.reference b/tests/queries/0_stateless/01780_dict_get_or_null.reference new file mode 100644 index 00000000000..4baca9ec91b --- /dev/null +++ b/tests/queries/0_stateless/01780_dict_get_or_null.reference @@ -0,0 +1,18 @@ +Simple key dictionary dictGetOrNull +0 0 \N \N (NULL,NULL) +1 1 First First ('First','First') +2 1 Second \N ('Second',NULL) +3 1 Third Third ('Third','Third') +4 0 \N \N (NULL,NULL) +Complex key dictionary dictGetOrNull +(0,'key') 0 \N \N (NULL,NULL) +(1,'key') 1 First First ('First','First') +(2,'key') 1 Second \N ('Second',NULL) +(3,'key') 1 Third Third ('Third','Third') +(4,'key') 0 \N \N (NULL,NULL) +Range key dictionary dictGetOrNull +(0,'2019-05-20') 0 \N \N (NULL,NULL) +(1,'2019-05-20') 1 First First ('First','First') +(2,'2019-05-20') 1 Second \N ('Second',NULL) +(3,'2019-05-20') 1 Third Third ('Third','Third') +(4,'2019-05-20') 0 \N \N (NULL,NULL) diff --git a/tests/queries/0_stateless/01780_dict_get_or_null.sql b/tests/queries/0_stateless/01780_dict_get_or_null.sql new file mode 100644 index 00000000000..f13bcf57d27 --- /dev/null +++ b/tests/queries/0_stateless/01780_dict_get_or_null.sql @@ -0,0 +1,116 @@ +DROP TABLE IF EXISTS simple_key_dictionary_source_table; +CREATE TABLE simple_key_dictionary_source_table +( + id UInt64, + value String, + value_nullable Nullable(String) +) ENGINE = TinyLog; + +INSERT INTO simple_key_dictionary_source_table VALUES (1, 'First', 'First'); +INSERT INTO simple_key_dictionary_source_table VALUES (2, 'Second', NULL); +INSERT INTO simple_key_dictionary_source_table VALUES (3, 'Third', 'Third'); + +DROP DICTIONARY IF EXISTS simple_key_dictionary; +CREATE DICTIONARY simple_key_dictionary +( + id UInt64, + value String, + value_nullable Nullable(String) +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_dictionary_source_table')) +LAYOUT(DIRECT()); + +SELECT 'Simple key dictionary dictGetOrNull'; + +SELECT + number, + dictHas('simple_key_dictionary', number), + dictGetOrNull('simple_key_dictionary', 'value', number), + dictGetOrNull('simple_key_dictionary', 'value_nullable', number), + dictGetOrNull('simple_key_dictionary', ('value', 'value_nullable'), number) +FROM system.numbers LIMIT 5; + +DROP DICTIONARY simple_key_dictionary; +DROP TABLE simple_key_dictionary_source_table; + +DROP TABLE IF EXISTS complex_key_dictionary_source_table; +CREATE TABLE complex_key_dictionary_source_table +( + id UInt64, + id_key String, + value String, + value_nullable Nullable(String) +) ENGINE = TinyLog; + +INSERT INTO complex_key_dictionary_source_table VALUES (1, 'key', 'First', 'First'); +INSERT INTO complex_key_dictionary_source_table VALUES (2, 'key', 'Second', NULL); +INSERT INTO complex_key_dictionary_source_table VALUES (3, 'key', 'Third', 'Third'); + +DROP DICTIONARY IF EXISTS complex_key_dictionary; +CREATE DICTIONARY complex_key_dictionary +( + id UInt64, + id_key String, + value String, + value_nullable Nullable(String) +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'complex_key_dictionary_source_table')) +LAYOUT(COMPLEX_KEY_DIRECT()); + +SELECT 'Complex key dictionary dictGetOrNull'; + +SELECT + (number, 'key'), + dictHas('complex_key_dictionary', (number, 'key')), + dictGetOrNull('complex_key_dictionary', 'value', (number, 'key')), + dictGetOrNull('complex_key_dictionary', 'value_nullable', (number, 'key')), + dictGetOrNull('complex_key_dictionary', ('value', 'value_nullable'), (number, 'key')) +FROM system.numbers LIMIT 5; + +DROP DICTIONARY complex_key_dictionary; +DROP TABLE complex_key_dictionary_source_table; + +DROP TABLE IF EXISTS range_key_dictionary_source_table; +CREATE TABLE range_key_dictionary_source_table +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +ENGINE = TinyLog(); + +INSERT INTO range_key_dictionary_source_table VALUES(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First'); +INSERT INTO range_key_dictionary_source_table VALUES(2, toDate('2019-05-20'), toDate('2019-05-20'), 'Second', NULL); +INSERT INTO range_key_dictionary_source_table VALUES(3, toDate('2019-05-20'), toDate('2019-05-20'), 'Third', 'Third'); + +DROP DICTIONARY IF EXISTS range_key_dictionary; +CREATE DICTIONARY range_key_dictionary +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_key_dictionary_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED()) +RANGE(MIN start_date MAX end_date); + +SELECT 'Range key dictionary dictGetOrNull'; + +SELECT + (number, toDate('2019-05-20')), + dictHas('range_key_dictionary', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value_nullable', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', ('value', 'value_nullable'), number, toDate('2019-05-20')) +FROM system.numbers LIMIT 5; + +DROP DICTIONARY range_key_dictionary; +DROP TABLE range_key_dictionary_source_table; From 2145fa3f21917a011971e471b16b46f393a0abe2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 31 Mar 2021 18:15:09 +0300 Subject: [PATCH 159/266] fix --- .../0_stateless/01666_merge_tree_max_query_limit.reference | 1 - tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference index a08a20dc95d..25880a7d740 100644 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference @@ -12,4 +12,3 @@ Check if another query is passed Modify max_concurrent_queries back to 1 Check if another query with less marks to read is throttled yes -finished long_running_query default select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index 11a68cf6607..5bb93371483 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -63,7 +63,7 @@ CODE=$? [ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; echo "yes" -${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '$query_id' SYNC" +${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '$query_id' SYNC FORMAT Null" wait ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " From 20066d8cd905539eb2cc908ebaccc626955f3eb3 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 31 Mar 2021 18:17:44 +0300 Subject: [PATCH 160/266] Update ExpressionElementParsers.cpp --- src/Parsers/ExpressionElementParsers.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index a15dbf3ea25..84c178790b2 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -49,7 +49,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; } From f4ab8b84b6186dd852c0aae6e9d067293de872d1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 31 Mar 2021 22:56:30 +0300 Subject: [PATCH 161/266] Added comment --- src/Functions/FunctionsExternalDictionaries.h | 47 ++++++++++++------- 1 file changed, 31 insertions(+), 16 deletions(-) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 92622b78cab..121e693c759 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -760,31 +760,30 @@ private: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - auto dictionary_name_argument_column = arguments[0]; - auto keys_argument_column = arguments[2]; + /** We call dictHas function to get which map is key presented in dictionary. + For key that presented in dictionary dict has result for that key index value will be 1. Otherwise 0. + We invert result, and then for key that is not presented in dictionary value will be 1. Otherwise 0. + This inverted result will be used as null column map. + After that we call dict get function, by contract for key that are not presented in dictionary we + return default value. + We create nullable column from dict get result column and null column map. - ColumnsWithTypeAndName dict_has_arguments; - dict_has_arguments.reserve(arguments.size() - 1); - - for (size_t i = 0; i < arguments.size(); ++i) - { - if (i == 1) - continue; - - dict_has_arguments.emplace_back(arguments[i]); - } + 2 additional implementation details: + 1. Result from dict get can be tuple if client requested multiple attributes we apply such operation on each result column. + 2. If column is already nullable we merge column null map with null map that we get from dict has. + */ + auto dict_has_arguments = filterAttributesForDictHas(arguments); auto is_key_in_dictionary_column = dictionary_has_func_impl.executeImpl(dict_has_arguments, std::make_shared(), input_rows_count); auto is_key_in_dictionary_column_mutable = is_key_in_dictionary_column->assumeMutable(); ColumnVector & is_key_in_dictionary_column_typed = assert_cast &>(*is_key_in_dictionary_column_mutable); - auto & is_key_in_dictionary_data = is_key_in_dictionary_column_typed.getData(); + PaddedPODArray & is_key_in_dictionary_data = is_key_in_dictionary_column_typed.getData(); + for (auto & key : is_key_in_dictionary_data) + key = !key; auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments); auto dictionary_get_result_column = dictionary_get_func_impl.executeImpl(arguments, result_type, input_rows_count); - for (auto & key : is_key_in_dictionary_data) - key = !key; - ColumnPtr result; WhichDataType result_data_type(result_type); @@ -840,6 +839,22 @@ private: null_map[i] = null_map[i] || null_map_to_add[i]; } + static ColumnsWithTypeAndName filterAttributesForDictHas(const ColumnsWithTypeAndName & arguments) + { + ColumnsWithTypeAndName dict_has_arguments; + dict_has_arguments.reserve(arguments.size() - 1); + + for (size_t i = 0; i < arguments.size(); ++i) + { + if (i == 1) + continue; + + dict_has_arguments.emplace_back(arguments[i]); + } + + return dict_has_arguments; + } + const FunctionDictGetNoType dictionary_get_func_impl; const FunctionDictHas dictionary_has_func_impl; }; From 2ad29a541a806589bcadac46e474ffcf8e8258d2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 1 Apr 2021 00:12:21 +0300 Subject: [PATCH 162/266] Updated interfaces --- src/Dictionaries/CacheDictionary.cpp | 51 +++++++------- src/Dictionaries/CacheDictionary.h | 6 -- src/Dictionaries/CacheDictionaryUpdateQueue.h | 3 +- src/Dictionaries/DictionaryHelpers.h | 22 ++++++ src/Dictionaries/DirectDictionary.cpp | 38 ++++------- src/Dictionaries/HashedDictionary.cpp | 67 ++++++++----------- tests/performance/hashed_dictionary.xml | 6 ++ 7 files changed, 94 insertions(+), 99 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 85f6cb98e8c..535e862af40 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -137,23 +137,6 @@ Columns CacheDictionary::getColumns( 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); - - Arena complex_keys_arena; - DictionaryKeysExtractor extractor(key_columns, &complex_keys_arena); - auto keys = extractor.extractAllKeys(); - - return getColumnsImpl(attribute_names, key_columns, keys, default_values_columns); -} - -template -Columns CacheDictionary::getColumnsImpl( - const Strings & attribute_names, - const Columns & key_columns, - const PaddedPODArray & keys, - const Columns & default_values_columns) const { /** * Flow of getColumsImpl @@ -169,6 +152,13 @@ Columns CacheDictionary::getColumnsImpl( * use default value. */ + if (dictionary_key_type == DictionaryKeyType::complex) + dict_struct.validateKeyTypes(key_types); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + auto keys = extractor.extractAllKeys(); + DictionaryStorageFetchRequest request(dict_struct, attribute_names, default_values_columns); FetchResult result_of_fetch_from_storage; @@ -281,8 +271,9 @@ ColumnUInt8::Ptr CacheDictionary::hasKeys(const Columns & k if (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena complex_keys_arena; - DictionaryKeysExtractor extractor(key_columns, &complex_keys_arena); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); const auto keys = extractor.extractAllKeys(); /// We make empty request just to fetch if keys exists @@ -531,14 +522,20 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr requested_keys_extractor(update_unit_ptr->key_columns, &update_unit_ptr->complex_key_arena); - const auto requested_keys = requested_keys_extractor.extractAllKeys(); + Arena * complex_key_arena = update_unit_ptr->complex_keys_arena_holder.getComplexKeyArena(); + DictionaryKeysExtractor requested_keys_extractor(update_unit_ptr->key_columns, complex_key_arena); + auto requested_keys = requested_keys_extractor.extractAllKeys(); HashSet not_found_keys; std::vector requested_keys_vector; std::vector requested_complex_key_rows; + if constexpr (dictionary_key_type == DictionaryKeyType::simple) + requested_keys_vector.reserve(requested_keys.size()); + else + requested_complex_key_rows.reserve(requested_keys.size()); + auto & key_index_to_state_from_storage = update_unit_ptr->key_index_to_state; for (size_t i = 0; i < key_index_to_state_from_storage.size(); ++i) @@ -598,26 +595,24 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr keys_extractor(key_columns, &update_unit_ptr->complex_key_arena); - const size_t keys_extracted_from_block_size = keys_extractor.getKeysSize(); + DictionaryKeysExtractor keys_extractor(key_columns, complex_key_arena); + auto keys_extracted_from_block = keys_extractor.extractAllKeys(); for (size_t index_of_attribute = 0; index_of_attribute < fetched_columns_during_update.size(); ++index_of_attribute) { auto & column_to_update = fetched_columns_during_update[index_of_attribute]; auto column = block.safeGetByPosition(skip_keys_size_offset + index_of_attribute).column; - column_to_update->assumeMutable()->insertRangeFrom(*column, 0, keys_extracted_from_block_size); + column_to_update->assumeMutable()->insertRangeFrom(*column, 0, keys_extracted_from_block.size()); } - for (size_t i = 0; i < keys_extracted_from_block_size; ++i) + for (size_t i = 0; i < keys_extracted_from_block.size(); ++i) { - auto fetched_key_from_source = keys_extractor.extractCurrentKey(); + auto fetched_key_from_source = keys_extracted_from_block[i]; not_found_keys.erase(fetched_key_from_source); update_unit_ptr->requested_keys_to_fetched_columns_during_update_index[fetched_key_from_source] = found_keys_size; found_keys_in_source.emplace_back(fetched_key_from_source); ++found_keys_size; - - keys_extractor.rollbackCurrentKey(); } } diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 35ea17abf27..62cd509d006 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -142,12 +142,6 @@ public: private: using FetchResult = std::conditional_t; - Columns getColumnsImpl( - const Strings & attribute_names, - const Columns & key_columns, - const PaddedPODArray & keys, - const Columns & default_values_columns) const; - static MutableColumns aggregateColumnsInOrderOfKeys( const PaddedPODArray & keys, const DictionaryStorageFetchRequest & request, diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.h b/src/Dictionaries/CacheDictionaryUpdateQueue.h index 2e636af6db6..3d27a157752 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.h +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.h @@ -66,8 +66,9 @@ public: HashMap requested_keys_to_fetched_columns_during_update_index; MutableColumns fetched_columns_during_update; + /// Complex keys are serialized in this arena - Arena complex_key_arena; + DictionaryKeysArenaHolder complex_keys_arena_holder; private: template diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 0be1ebb7728..3e7063bb9ef 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -295,6 +295,28 @@ private: bool use_default_value_from_column = false; }; +template +class DictionaryKeysArenaHolder; + +template <> +class DictionaryKeysArenaHolder +{ +public: + static Arena * getComplexKeyArena() { return nullptr; } +}; + +template <> +class DictionaryKeysArenaHolder +{ +public: + + Arena * getComplexKeyArena() { return &complex_key_arena; } + +private: + Arena complex_key_arena; +}; + + template class DictionaryKeysExtractor { diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 87af00e56a6..47698949924 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -41,15 +41,12 @@ ColumnPtr DirectDictionary::getColumn( if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - - DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); const auto requested_keys = extractor.extractAllKeys(); + Field value; + const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type); DefaultValueProvider default_value_provider(attribute.null_value, default_values_column); @@ -79,20 +76,18 @@ ColumnPtr DirectDictionary::getColumn( block_columns.erase(block_columns.begin()); } - DictionaryKeysExtractor block_keys_extractor(block_key_columns, temporary_complex_key_arena); - size_t block_keys_size = block_keys_extractor.getKeysSize(); + DictionaryKeysExtractor block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena()); + auto block_keys = block_keys_extractor.extractAllKeys(); const auto & block_column = block.safeGetByPosition(dictionary_keys_size + requested_attribute_index).column; - fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys_size); + fetched_from_storage->insertRangeFrom(*block_column, 0, block_keys.size()); - for (size_t block_key_index = 0; block_key_index < block_keys_size; ++block_key_index) + for (size_t block_key_index = 0; block_key_index < block_keys.size(); ++block_key_index) { - const auto block_key = block_keys_extractor.extractCurrentKey(); + auto block_key = block_keys[block_key_index]; key_to_fetched_index[block_key] = fetched_key_index; ++fetched_key_index; - - block_keys_extractor.rollbackCurrentKey(); } block_key_columns.clear(); @@ -131,15 +126,10 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & if constexpr (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - - DictionaryKeysExtractor requested_keys_extractor(key_columns, temporary_complex_key_arena); - const auto requested_keys = requested_keys_extractor.extractAllKeys(); - size_t requested_keys_size = requested_keys_extractor.getKeysSize(); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor requested_keys_extractor(key_columns, arena_holder.getComplexKeyArena()); + auto requested_keys = requested_keys_extractor.extractAllKeys(); + size_t requested_keys_size = requested_keys.size(); HashMap requested_key_to_index; requested_key_to_index.reserve(requested_keys_size); @@ -172,7 +162,7 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & block_columns.erase(block_columns.begin()); } - DictionaryKeysExtractor block_keys_extractor(block_key_columns, temporary_complex_key_arena); + DictionaryKeysExtractor block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena()); size_t block_keys_size = block_keys_extractor.getKeysSize(); for (size_t i = 0; i < block_keys_size; ++i) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 37520e5a2ec..d45e4ade1cf 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -72,13 +72,8 @@ ColumnPtr HashedDictionary::getColumn( ColumnPtr result; - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - - DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); const size_t size = extractor.getKeysSize(); @@ -115,7 +110,11 @@ ColumnPtr HashedDictionary::getColumn( attribute, extractor, [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, - [&](const size_t row) { (*vec_null_map_to)[row] = true; }, + [&](const size_t row) + { + out->insertDefault(); + (*vec_null_map_to)[row] = true; + }, default_value_extractor); } else @@ -126,7 +125,11 @@ ColumnPtr HashedDictionary::getColumn( attribute, extractor, [&](const size_t row, const auto value) { return out[row] = value; }, - [&](const size_t row) { (*vec_null_map_to)[row] = true; }, + [&](const size_t row) + { + out[row] = 0; + (*vec_null_map_to)[row] = true; + }, default_value_extractor); } @@ -147,13 +150,8 @@ ColumnUInt8::Ptr HashedDictionary::hasKeys(const Co if (dictionary_key_type == DictionaryKeyType::complex) dict_struct.validateKeyTypes(key_types); - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - - DictionaryKeysExtractor extractor(key_columns, temporary_complex_key_arena); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); size_t keys_size = extractor.getKeysSize(); @@ -381,14 +379,9 @@ void HashedDictionary::updateData() saved_block_key_columns.emplace_back(saved_block->safeGetByPosition(i).column); - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - - DictionaryKeysExtractor saved_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); - const auto & saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys(); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor saved_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena()); + auto saved_keys_extracted_from_block = saved_keys_extractor.extractAllKeys(); auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); @@ -403,15 +396,14 @@ void HashedDictionary::updateData() for (size_t i = 0; i < skip_keys_size_offset; ++i) block_key_columns.emplace_back(block.safeGetByPosition(i).column); - DictionaryKeysExtractor block_keys_extractor(saved_block_key_columns, temporary_complex_key_arena); - const auto keys_extracted_from_block_size = block_keys_extractor.getKeysSize(); + DictionaryKeysExtractor block_keys_extractor(saved_block_key_columns, arena_holder.getComplexKeyArena()); + auto keys_extracted_from_block = block_keys_extractor.extractAllKeys(); absl::flat_hash_map, DefaultHash> update_keys; - for (size_t row = 0; row < keys_extracted_from_block_size; ++row) + for (size_t row = 0; row < keys_extracted_from_block.size(); ++row) { - const auto key = block_keys_extractor.extractCurrentKey(); + auto key = keys_extracted_from_block[row]; update_keys[key].push_back(row); - block_keys_extractor.rollbackCurrentKey(); } IColumn::Filter filter(saved_keys_extracted_from_block.size()); @@ -447,12 +439,6 @@ void HashedDictionary::updateData() template void HashedDictionary::blockToAttributes(const Block & block [[maybe_unused]]) { - Arena temporary_complex_keys_arena_holder [[maybe_unused]]; - Arena * temporary_complex_key_arena = nullptr; - - if constexpr (dictionary_key_type == DictionaryKeyType::complex) - temporary_complex_key_arena = &temporary_complex_keys_arena_holder; - size_t skip_keys_size_offset = dict_struct.getKeysSize(); Columns key_columns; @@ -462,7 +448,8 @@ void HashedDictionary::blockToAttributes(const Bloc for (size_t i = 0; i < skip_keys_size_offset; ++i) key_columns.emplace_back(block.safeGetByPosition(i).column); - DictionaryKeysExtractor keys_extractor(key_columns, temporary_complex_key_arena); + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); const size_t keys_size = keys_extractor.getKeysSize(); Field column_value_to_insert; @@ -561,7 +548,7 @@ void HashedDictionary::getItemsImpl( const auto & attribute_container = std::get>(attribute.container); const size_t keys_size = keys_extractor.getKeysSize(); - // bool is_attribute_nullable = attribute.is_nullable_set.has_value(); + bool is_attribute_nullable = attribute.is_nullable_set.has_value(); for (size_t key_index = 0; key_index < keys_size; ++key_index) { @@ -573,9 +560,9 @@ void HashedDictionary::getItemsImpl( set_value(key_index, getValueFromCell(it)); else { - // if (is_attribute_nullable && attribute.is_nullable_set->find(key) != nullptr) - // set_nullable_value(key_index); - // else + if (is_attribute_nullable && attribute.is_nullable_set->find(key) != nullptr) + set_nullable_value(key_index); + else set_value(key_index, default_value_extractor[key_index]); } diff --git a/tests/performance/hashed_dictionary.xml b/tests/performance/hashed_dictionary.xml index 150b0309ce4..a38d2f30c23 100644 --- a/tests/performance/hashed_dictionary.xml +++ b/tests/performance/hashed_dictionary.xml @@ -108,6 +108,12 @@ LIMIT {elements_count} FORMAT Null; + + SELECT dictHas('default.complex_key_hashed_dictionary', (number, toString(number))) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + DROP TABLE IF EXISTS simple_key_hashed_dictionary_source_table; DROP TABLE IF EXISTS complex_key_hashed_dictionary_source_table; From 8176a94e84ebdd1ab8d3c17feeea9473f7a46243 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Thu, 1 Apr 2021 01:30:23 +0300 Subject: [PATCH 163/266] Update docs/ru/engines/database-engines/postgresql.md Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- docs/ru/engines/database-engines/postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/database-engines/postgresql.md b/docs/ru/engines/database-engines/postgresql.md index afda3feec5a..c11dab6f1aa 100644 --- a/docs/ru/engines/database-engines/postgresql.md +++ b/docs/ru/engines/database-engines/postgresql.md @@ -9,7 +9,7 @@ toc_title: PostgreSQL Позволяет в реальном времени получать от удаленного сервера PostgreSQL информацию о таблицах БД и их структуре с помощью запросов `SHOW TABLES` и `DESCRIBE TABLE`. -Поддерживает операции изменения структуры таблиц (`ALTER TABLE ... ADD|DROP COLUMN`). Если параметр `use_table_cache` (см. ниже раздел Параметры движка) установлен в значение `1`, структура таблицы кешируется, и изменения в структуре не отображаются мгновенно, но могут быть получены с помощью запросов `DETACH` и `ATTACH`. +Поддерживает операции изменения структуры таблиц (`ALTER TABLE ... ADD|DROP COLUMN`). Если параметр `use_table_cache` (см. ниже раздел Параметры движка) установлен в значение `1`, структура таблицы кешируется, и изменения в структуре не отслеживаются, но будут обновлены, если выполнить команды `DETACH` и `ATTACH`. ## Создание БД {#creating-a-database} From 3fbe407aaaf43009536a329d05f130b2f9886ba8 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Thu, 1 Apr 2021 01:48:48 +0300 Subject: [PATCH 164/266] Replicas priority example added. --- .../table-engines/integrations/postgresql.md | 25 +++++++++++++++++- .../table-engines/integrations/postgresql.md | 26 ++++++++++++++++++- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 7bf930d4120..ad5bebb3dea 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -50,7 +50,30 @@ PostgreSQL `Array` types are converted into ClickHouse arrays. !!! info "Note" Be careful - in PostgreSQL an array data, created like a `type_name[]`, may contain multi-dimensional arrays of different dimensions in different table rows in same column. But in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. -Supports replicas priority for PostgreSQL dictionary source. The bigger the number in map, the less the priority. The highest priority is `0`. +Replicas priority for PostgreSQL dictionary source is supported. The bigger the number in map, the less the priority. The highest priority is `0`. + +In the example below replica `example01-1` has the highest priority: + +```xml + + 5432 + clickhouse + qwerty + + example01-1 + 1 + + + example01-2 + 2 + + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` ## Usage Example {#usage-example} diff --git a/docs/ru/engines/table-engines/integrations/postgresql.md b/docs/ru/engines/table-engines/integrations/postgresql.md index 0498a9e12a5..8964b1dbf02 100644 --- a/docs/ru/engines/table-engines/integrations/postgresql.md +++ b/docs/ru/engines/table-engines/integrations/postgresql.md @@ -50,7 +50,31 @@ PostgreSQL массивы конвертируются в массивы ClickHo !!! info "Внимание" Будьте внимательны, в PostgreSQL массивы, созданные как `type_name[]`, являются многомерными и могут содержать в себе разное количество измерений в разных строках одной таблицы. Внутри ClickHouse допустимы только многомерные массивы с одинаковым кол-вом измерений во всех строках таблицы. -При использовании словаря PostgreSQL поддерживается приоритет реплик. Чем больше номер реплики, тем ниже ее приоритет. Наивысший приоритет у реплики с номером `0`. +При использовании словаря PostgreSQL поддерживается приоритет реплик. Чем больше номер реплики, тем ниже ее приоритет. Наивысший приоритет у реплики с номером `0`. + +В примере ниже реплика `example01-1` имеет более высокий приоритет: + +```xml + + 5432 + clickhouse + qwerty + + example01-1 + 1 + + + example01-2 + 2 + + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` + ## Пример использования {#usage-example} Таблица в PostgreSQL: From 0b956b7d978abd836a37cf89d619679540cd50a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 02:35:41 +0300 Subject: [PATCH 165/266] Remove unsupported languages #20972 --- docs/es/commercial/cloud.md | 23 - docs/es/commercial/index.md | 9 - docs/es/commercial/support.md | 23 - docs/es/development/architecture.md | 203 -- docs/es/development/browse-code.md | 14 - docs/es/development/build-cross-arm.md | 43 - docs/es/development/build-cross-osx.md | 64 - docs/es/development/build-osx.md | 93 - docs/es/development/build.md | 141 -- docs/es/development/contrib.md | 41 - docs/es/development/developer-instruction.md | 287 --- docs/es/development/index.md | 12 - docs/es/development/style.md | 841 -------- docs/es/development/tests.md | 1 - docs/es/engines/database-engines/atomic.md | 17 - docs/es/engines/database-engines/index.md | 21 - docs/es/engines/database-engines/lazy.md | 18 - docs/es/engines/database-engines/mysql.md | 135 -- docs/es/engines/index.md | 8 - docs/es/engines/table-engines/index.md | 85 - .../table-engines/integrations/hdfs.md | 123 -- .../table-engines/integrations/index.md | 8 - .../table-engines/integrations/jdbc.md | 90 - .../table-engines/integrations/kafka.md | 180 -- .../table-engines/integrations/mysql.md | 105 - .../table-engines/integrations/odbc.md | 132 -- .../engines/table-engines/log-family/index.md | 47 - .../engines/table-engines/log-family/log.md | 16 - .../table-engines/log-family/stripelog.md | 95 - .../table-engines/log-family/tinylog.md | 16 - .../mergetree-family/aggregatingmergetree.md | 105 - .../mergetree-family/collapsingmergetree.md | 306 --- .../custom-partitioning-key.md | 127 -- .../mergetree-family/graphitemergetree.md | 174 -- .../table-engines/mergetree-family/index.md | 8 - .../mergetree-family/mergetree.md | 654 ------ .../mergetree-family/replacingmergetree.md | 69 - .../mergetree-family/replication.md | 218 -- .../mergetree-family/summingmergetree.md | 141 -- .../versionedcollapsingmergetree.md | 238 -- .../engines/table-engines/special/buffer.md | 71 - .../table-engines/special/dictionary.md | 97 - .../table-engines/special/distributed.md | 152 -- .../table-engines/special/external-data.md | 68 - docs/es/engines/table-engines/special/file.md | 90 - .../engines/table-engines/special/generate.md | 61 - .../es/engines/table-engines/special/index.md | 8 - docs/es/engines/table-engines/special/join.md | 111 - .../table-engines/special/materializedview.md | 12 - .../engines/table-engines/special/memory.md | 19 - .../es/engines/table-engines/special/merge.md | 70 - docs/es/engines/table-engines/special/null.md | 14 - docs/es/engines/table-engines/special/set.md | 19 - docs/es/engines/table-engines/special/url.md | 82 - docs/es/engines/table-engines/special/view.md | 12 - docs/es/faq/general.md | 60 - docs/es/faq/index.md | 8 - .../example-datasets/amplab-benchmark.md | 129 -- .../example-datasets/criteo.md | 81 - .../getting-started/example-datasets/index.md | 22 - .../example-datasets/metrica.md | 70 - .../example-datasets/nyc-taxi.md | 390 ---- .../example-datasets/ontime.md | 412 ---- .../example-datasets/star-schema.md | 370 ---- .../example-datasets/wikistat.md | 35 - docs/es/getting-started/index.md | 17 - docs/es/getting-started/install.md | 182 -- docs/es/getting-started/playground.md | 48 - docs/es/getting-started/tutorial.md | 664 ------ docs/es/guides/apply-catboost-model.md | 239 -- docs/es/guides/index.md | 16 - docs/es/images/column-oriented.gif | Bin 43771 -> 0 bytes docs/es/images/logo.svg | 1 - docs/es/images/row-oriented.gif | Bin 39281 -> 0 bytes docs/es/index.md | 97 - docs/es/interfaces/cli.md | 149 -- docs/es/interfaces/cpp.md | 12 - docs/es/interfaces/formats.md | 1212 ----------- docs/es/interfaces/http.md | 617 ------ docs/es/interfaces/index.md | 29 - docs/es/interfaces/jdbc.md | 15 - docs/es/interfaces/mysql.md | 49 - docs/es/interfaces/odbc.md | 12 - docs/es/interfaces/tcp.md | 12 - .../third-party/client-libraries.md | 60 - docs/es/interfaces/third-party/gui.md | 156 -- docs/es/interfaces/third-party/index.md | 8 - .../es/interfaces/third-party/integrations.md | 108 - docs/es/interfaces/third-party/proxy.md | 46 - docs/es/introduction/adopters.md | 86 - docs/es/introduction/distinctive-features.md | 77 - docs/es/introduction/history.md | 56 - docs/es/introduction/index.md | 8 - docs/es/introduction/performance.md | 32 - docs/es/operations/access-rights.md | 143 -- docs/es/operations/backup.md | 39 - docs/es/operations/configuration-files.md | 57 - docs/es/operations/index.md | 28 - docs/es/operations/monitoring.md | 46 - .../optimizing-performance/index.md | 8 - .../sampling-query-profiler.md | 64 - docs/es/operations/performance-test.md | 82 - docs/es/operations/quotas.md | 112 - docs/es/operations/requirements.md | 61 - .../server-configuration-parameters/index.md | 19 - .../settings.md | 906 -------- .../settings/constraints-on-settings.md | 75 - docs/es/operations/settings/index.md | 33 - .../settings/permissions-for-queries.md | 61 - .../operations/settings/query-complexity.md | 300 --- .../operations/settings/settings-profiles.md | 81 - docs/es/operations/settings/settings-users.md | 164 -- docs/es/operations/settings/settings.md | 1254 ----------- docs/es/operations/system-tables.md | 1168 ---------- docs/es/operations/tips.md | 251 --- docs/es/operations/troubleshooting.md | 146 -- docs/es/operations/update.md | 20 - .../utilities/clickhouse-benchmark.md | 156 -- .../operations/utilities/clickhouse-copier.md | 176 -- .../operations/utilities/clickhouse-local.md | 81 - docs/es/operations/utilities/index.md | 15 - docs/es/roadmap.md | 16 - .../aggregate-functions/combinators.md | 245 --- .../aggregate-functions/index.md | 62 - .../parametric-functions.md | 499 ----- .../aggregate-functions/reference.md | 1914 ----------------- docs/es/sql-reference/ansi.md | 180 -- .../data-types/aggregatefunction.md | 70 - docs/es/sql-reference/data-types/array.md | 77 - docs/es/sql-reference/data-types/boolean.md | 12 - docs/es/sql-reference/data-types/date.md | 14 - docs/es/sql-reference/data-types/datetime.md | 129 -- .../es/sql-reference/data-types/datetime64.md | 104 - docs/es/sql-reference/data-types/decimal.md | 109 - .../sql-reference/data-types/domains/index.md | 33 - .../sql-reference/data-types/domains/ipv4.md | 84 - .../sql-reference/data-types/domains/ipv6.md | 86 - docs/es/sql-reference/data-types/enum.md | 132 -- .../sql-reference/data-types/fixedstring.md | 63 - docs/es/sql-reference/data-types/float.md | 87 - docs/es/sql-reference/data-types/index.md | 15 - docs/es/sql-reference/data-types/int-uint.md | 26 - .../nested-data-structures/index.md | 12 - .../nested-data-structures/nested.md | 106 - docs/es/sql-reference/data-types/nullable.md | 46 - .../data-types/simpleaggregatefunction.md | 33 - .../special-data-types/expression.md | 12 - .../data-types/special-data-types/index.md | 14 - .../data-types/special-data-types/interval.md | 85 - .../data-types/special-data-types/nothing.md | 26 - .../data-types/special-data-types/set.md | 12 - docs/es/sql-reference/data-types/string.md | 20 - docs/es/sql-reference/data-types/tuple.md | 52 - docs/es/sql-reference/data-types/uuid.md | 77 - .../external-dicts-dict-hierarchical.md | 70 - .../external-dicts-dict-layout.md | 396 ---- .../external-dicts-dict-lifetime.md | 91 - .../external-dicts-dict-sources.md | 630 ------ .../external-dicts-dict-structure.md | 175 -- .../external-dicts-dict.md | 53 - .../external-dictionaries/external-dicts.md | 62 - .../external-dictionaries/index.md | 8 - docs/es/sql-reference/dictionaries/index.md | 22 - .../dictionaries/internal-dicts.md | 55 - .../functions/arithmetic-functions.md | 87 - .../functions/array-functions.md | 1061 --------- docs/es/sql-reference/functions/array-join.md | 37 - .../sql-reference/functions/bit-functions.md | 255 --- .../functions/bitmap-functions.md | 496 ----- .../functions/comparison-functions.md | 37 - .../functions/conditional-functions.md | 207 -- .../functions/date-time-functions.md | 450 ---- .../functions/encoding-functions.md | 175 -- .../functions/ext-dict-functions.md | 205 -- .../functions/functions-for-nulls.md | 312 --- docs/es/sql-reference/functions/geo.md | 510 ----- .../sql-reference/functions/hash-functions.md | 484 ----- .../functions/higher-order-functions.md | 264 --- .../sql-reference/functions/in-functions.md | 26 - docs/es/sql-reference/functions/index.md | 74 - .../sql-reference/functions/introspection.md | 310 --- .../functions/ip-address-functions.md | 248 --- .../sql-reference/functions/json-functions.md | 297 --- .../functions/logical-functions.md | 22 - .../functions/machine-learning-functions.md | 20 - .../sql-reference/functions/math-functions.md | 116 - .../functions/other-functions.md | 1205 ----------- .../functions/random-functions.md | 65 - .../functions/rounding-functions.md | 190 -- .../functions/splitting-merging-functions.md | 116 - .../functions/string-functions.md | 489 ----- .../functions/string-replace-functions.md | 94 - .../functions/string-search-functions.md | 383 ---- .../functions/type-conversion-functions.md | 534 ----- .../sql-reference/functions/url-functions.md | 209 -- .../sql-reference/functions/uuid-functions.md | 122 -- .../functions/ym-dict-functions.md | 155 -- docs/es/sql-reference/index.md | 20 - docs/es/sql-reference/operators/in.md | 204 -- docs/es/sql-reference/operators/index.md | 277 --- docs/es/sql-reference/statements/alter.md | 602 ------ docs/es/sql-reference/statements/create.md | 502 ----- docs/es/sql-reference/statements/grant.md | 476 ---- docs/es/sql-reference/statements/index.md | 8 - .../sql-reference/statements/insert-into.md | 80 - docs/es/sql-reference/statements/misc.md | 358 --- docs/es/sql-reference/statements/revoke.md | 50 - .../statements/select/array-join.md | 282 --- .../statements/select/distinct.md | 63 - .../sql-reference/statements/select/format.md | 18 - .../sql-reference/statements/select/from.md | 44 - .../statements/select/group-by.md | 132 -- .../sql-reference/statements/select/having.md | 14 - .../sql-reference/statements/select/index.md | 158 -- .../statements/select/into-outfile.md | 14 - .../sql-reference/statements/select/join.md | 187 -- .../statements/select/limit-by.md | 71 - .../sql-reference/statements/select/limit.md | 14 - .../statements/select/order-by.md | 72 - .../statements/select/prewhere.md | 22 - .../sql-reference/statements/select/sample.md | 113 - .../sql-reference/statements/select/union.md | 35 - .../sql-reference/statements/select/where.md | 15 - .../sql-reference/statements/select/with.md | 80 - docs/es/sql-reference/statements/show.md | 169 -- docs/es/sql-reference/statements/system.md | 113 - docs/es/sql-reference/syntax.md | 187 -- docs/es/sql-reference/table-functions/file.md | 121 -- .../sql-reference/table-functions/generate.md | 44 - docs/es/sql-reference/table-functions/hdfs.md | 104 - .../es/sql-reference/table-functions/index.md | 38 - .../es/sql-reference/table-functions/input.md | 47 - docs/es/sql-reference/table-functions/jdbc.md | 29 - .../es/sql-reference/table-functions/merge.md | 14 - .../es/sql-reference/table-functions/mysql.md | 86 - .../sql-reference/table-functions/numbers.md | 30 - docs/es/sql-reference/table-functions/odbc.md | 108 - .../sql-reference/table-functions/remote.md | 85 - docs/es/sql-reference/table-functions/url.md | 26 - docs/es/whats-new/changelog/2017.md | 1 - docs/es/whats-new/changelog/2018.md | 1 - docs/es/whats-new/changelog/2019.md | 1 - docs/es/whats-new/changelog/index.md | 1 - docs/es/whats-new/index.md | 8 - docs/es/whats-new/roadmap.md | 19 - docs/es/whats-new/security-changelog.md | 76 - docs/fr/commercial/cloud.md | 23 - docs/fr/commercial/index.md | 9 - docs/fr/commercial/support.md | 23 - docs/fr/development/architecture.md | 203 -- docs/fr/development/browse-code.md | 14 - docs/fr/development/build-cross-arm.md | 43 - docs/fr/development/build-cross-osx.md | 64 - docs/fr/development/build-osx.md | 93 - docs/fr/development/build.md | 141 -- docs/fr/development/contrib.md | 41 - docs/fr/development/developer-instruction.md | 287 --- docs/fr/development/index.md | 12 - docs/fr/development/style.md | 841 -------- docs/fr/development/tests.md | 1 - docs/fr/engines/database-engines/atomic.md | 17 - docs/fr/engines/database-engines/index.md | 21 - docs/fr/engines/database-engines/lazy.md | 18 - docs/fr/engines/database-engines/mysql.md | 135 -- docs/fr/engines/index.md | 8 - docs/fr/engines/table-engines/index.md | 85 - .../table-engines/integrations/hdfs.md | 123 -- .../table-engines/integrations/index.md | 8 - .../table-engines/integrations/jdbc.md | 90 - .../table-engines/integrations/kafka.md | 180 -- .../table-engines/integrations/mysql.md | 105 - .../table-engines/integrations/odbc.md | 132 -- .../engines/table-engines/log-family/index.md | 46 - .../engines/table-engines/log-family/log.md | 16 - .../table-engines/log-family/stripelog.md | 95 - .../table-engines/log-family/tinylog.md | 16 - .../mergetree-family/aggregatingmergetree.md | 105 - .../mergetree-family/collapsingmergetree.md | 306 --- .../custom-partitioning-key.md | 127 -- .../mergetree-family/graphitemergetree.md | 174 -- .../table-engines/mergetree-family/index.md | 8 - .../mergetree-family/mergetree.md | 654 ------ .../mergetree-family/replacingmergetree.md | 69 - .../mergetree-family/replication.md | 218 -- .../mergetree-family/summingmergetree.md | 141 -- .../versionedcollapsingmergetree.md | 238 -- .../engines/table-engines/special/buffer.md | 71 - .../table-engines/special/dictionary.md | 97 - .../table-engines/special/distributed.md | 152 -- .../table-engines/special/external-data.md | 68 - docs/fr/engines/table-engines/special/file.md | 90 - .../engines/table-engines/special/generate.md | 61 - .../fr/engines/table-engines/special/index.md | 8 - docs/fr/engines/table-engines/special/join.md | 111 - .../table-engines/special/materializedview.md | 12 - .../engines/table-engines/special/memory.md | 19 - .../fr/engines/table-engines/special/merge.md | 70 - docs/fr/engines/table-engines/special/null.md | 14 - docs/fr/engines/table-engines/special/set.md | 19 - docs/fr/engines/table-engines/special/url.md | 82 - docs/fr/engines/table-engines/special/view.md | 12 - docs/fr/faq/general.md | 60 - docs/fr/faq/index.md | 8 - .../example-datasets/amplab-benchmark.md | 129 -- .../example-datasets/criteo.md | 81 - .../getting-started/example-datasets/index.md | 22 - .../example-datasets/metrica.md | 70 - .../example-datasets/nyc-taxi.md | 390 ---- .../example-datasets/ontime.md | 412 ---- .../example-datasets/star-schema.md | 370 ---- .../example-datasets/wikistat.md | 35 - docs/fr/getting-started/index.md | 17 - docs/fr/getting-started/install.md | 182 -- docs/fr/getting-started/playground.md | 48 - docs/fr/getting-started/tutorial.md | 664 ------ docs/fr/guides/apply-catboost-model.md | 239 -- docs/fr/guides/index.md | 16 - docs/fr/images | 1 - docs/fr/index.md | 97 - docs/fr/interfaces/cli.md | 149 -- docs/fr/interfaces/cpp.md | 12 - docs/fr/interfaces/formats.md | 1212 ----------- docs/fr/interfaces/http.md | 617 ------ docs/fr/interfaces/index.md | 29 - docs/fr/interfaces/jdbc.md | 15 - docs/fr/interfaces/mysql.md | 49 - docs/fr/interfaces/odbc.md | 12 - docs/fr/interfaces/tcp.md | 12 - .../third-party/client-libraries.md | 62 - docs/fr/interfaces/third-party/gui.md | 156 -- docs/fr/interfaces/third-party/index.md | 8 - .../fr/interfaces/third-party/integrations.md | 110 - docs/fr/interfaces/third-party/proxy.md | 46 - docs/fr/introduction/adopters.md | 86 - docs/fr/introduction/distinctive-features.md | 77 - docs/fr/introduction/history.md | 56 - docs/fr/introduction/index.md | 8 - docs/fr/introduction/performance.md | 32 - docs/fr/operations/access-rights.md | 143 -- docs/fr/operations/backup.md | 41 - docs/fr/operations/configuration-files.md | 57 - docs/fr/operations/index.md | 28 - docs/fr/operations/monitoring.md | 46 - .../optimizing-performance/index.md | 8 - .../sampling-query-profiler.md | 64 - docs/fr/operations/performance-test.md | 82 - docs/fr/operations/quotas.md | 112 - docs/fr/operations/requirements.md | 61 - .../server-configuration-parameters/index.md | 19 - .../settings.md | 906 -------- .../settings/constraints-on-settings.md | 75 - docs/fr/operations/settings/index.md | 33 - .../settings/permissions-for-queries.md | 61 - .../operations/settings/query-complexity.md | 301 --- .../operations/settings/settings-profiles.md | 81 - docs/fr/operations/settings/settings-users.md | 164 -- docs/fr/operations/settings/settings.md | 1254 ----------- docs/fr/operations/system-tables.md | 1168 ---------- docs/fr/operations/tips.md | 251 --- docs/fr/operations/troubleshooting.md | 146 -- docs/fr/operations/update.md | 20 - .../utilities/clickhouse-benchmark.md | 156 -- .../operations/utilities/clickhouse-copier.md | 176 -- .../operations/utilities/clickhouse-local.md | 81 - docs/fr/operations/utilities/index.md | 15 - .../aggregate-functions/combinators.md | 245 --- .../aggregate-functions/index.md | 62 - .../parametric-functions.md | 499 ----- .../aggregate-functions/reference.md | 1914 ----------------- docs/fr/sql-reference/ansi.md | 180 -- .../data-types/aggregatefunction.md | 70 - docs/fr/sql-reference/data-types/array.md | 77 - docs/fr/sql-reference/data-types/boolean.md | 12 - docs/fr/sql-reference/data-types/date.md | 14 - docs/fr/sql-reference/data-types/datetime.md | 129 -- .../fr/sql-reference/data-types/datetime64.md | 104 - docs/fr/sql-reference/data-types/decimal.md | 109 - .../sql-reference/data-types/domains/index.md | 33 - .../sql-reference/data-types/domains/ipv4.md | 84 - .../sql-reference/data-types/domains/ipv6.md | 86 - docs/fr/sql-reference/data-types/enum.md | 132 -- .../sql-reference/data-types/fixedstring.md | 63 - docs/fr/sql-reference/data-types/float.md | 87 - docs/fr/sql-reference/data-types/index.md | 15 - docs/fr/sql-reference/data-types/int-uint.md | 26 - .../nested-data-structures/index.md | 12 - .../nested-data-structures/nested.md | 106 - docs/fr/sql-reference/data-types/nullable.md | 46 - .../data-types/simpleaggregatefunction.md | 38 - .../special-data-types/expression.md | 12 - .../data-types/special-data-types/index.md | 14 - .../data-types/special-data-types/interval.md | 85 - .../data-types/special-data-types/nothing.md | 26 - .../data-types/special-data-types/set.md | 12 - docs/fr/sql-reference/data-types/string.md | 20 - docs/fr/sql-reference/data-types/tuple.md | 52 - docs/fr/sql-reference/data-types/uuid.md | 77 - .../external-dicts-dict-hierarchical.md | 70 - .../external-dicts-dict-layout.md | 407 ---- .../external-dicts-dict-lifetime.md | 91 - .../external-dicts-dict-sources.md | 630 ------ .../external-dicts-dict-structure.md | 175 -- .../external-dicts-dict.md | 53 - .../external-dictionaries/external-dicts.md | 62 - .../external-dictionaries/index.md | 8 - docs/fr/sql-reference/dictionaries/index.md | 22 - .../dictionaries/internal-dicts.md | 55 - .../functions/arithmetic-functions.md | 87 - .../functions/array-functions.md | 1061 --------- docs/fr/sql-reference/functions/array-join.md | 37 - .../sql-reference/functions/bit-functions.md | 255 --- .../functions/bitmap-functions.md | 496 ----- .../functions/comparison-functions.md | 37 - .../functions/conditional-functions.md | 207 -- .../functions/date-time-functions.md | 450 ---- .../functions/encoding-functions.md | 175 -- .../functions/ext-dict-functions.md | 205 -- .../functions/functions-for-nulls.md | 312 --- docs/fr/sql-reference/functions/geo.md | 510 ----- .../sql-reference/functions/hash-functions.md | 484 ----- .../functions/higher-order-functions.md | 264 --- .../sql-reference/functions/in-functions.md | 26 - docs/fr/sql-reference/functions/index.md | 74 - .../sql-reference/functions/introspection.md | 310 --- .../functions/ip-address-functions.md | 248 --- .../sql-reference/functions/json-functions.md | 297 --- .../functions/logical-functions.md | 22 - .../functions/machine-learning-functions.md | 20 - .../sql-reference/functions/math-functions.md | 116 - .../functions/other-functions.md | 1205 ----------- .../functions/random-functions.md | 65 - .../functions/rounding-functions.md | 190 -- .../functions/splitting-merging-functions.md | 116 - .../functions/string-functions.md | 489 ----- .../functions/string-replace-functions.md | 94 - .../functions/string-search-functions.md | 379 ---- .../functions/type-conversion-functions.md | 534 ----- .../sql-reference/functions/url-functions.md | 209 -- .../sql-reference/functions/uuid-functions.md | 122 -- .../functions/ym-dict-functions.md | 155 -- docs/fr/sql-reference/index.md | 20 - docs/fr/sql-reference/operators/in.md | 204 -- docs/fr/sql-reference/operators/index.md | 277 --- docs/fr/sql-reference/statements/alter.md | 602 ------ docs/fr/sql-reference/statements/create.md | 502 ----- docs/fr/sql-reference/statements/grant.md | 476 ---- docs/fr/sql-reference/statements/index.md | 8 - .../sql-reference/statements/insert-into.md | 80 - docs/fr/sql-reference/statements/misc.md | 358 --- docs/fr/sql-reference/statements/revoke.md | 50 - .../statements/select/array-join.md | 282 --- .../statements/select/distinct.md | 63 - .../sql-reference/statements/select/format.md | 18 - .../sql-reference/statements/select/from.md | 44 - .../statements/select/group-by.md | 132 -- .../sql-reference/statements/select/having.md | 14 - .../sql-reference/statements/select/index.md | 158 -- .../statements/select/into-outfile.md | 14 - .../sql-reference/statements/select/join.md | 187 -- .../statements/select/limit-by.md | 71 - .../sql-reference/statements/select/limit.md | 14 - .../statements/select/order-by.md | 72 - .../statements/select/prewhere.md | 22 - .../sql-reference/statements/select/sample.md | 113 - .../sql-reference/statements/select/union.md | 35 - .../sql-reference/statements/select/where.md | 15 - .../sql-reference/statements/select/with.md | 80 - docs/fr/sql-reference/statements/show.md | 169 -- docs/fr/sql-reference/statements/system.md | 113 - docs/fr/sql-reference/syntax.md | 187 -- docs/fr/sql-reference/table-functions/file.md | 121 -- .../sql-reference/table-functions/generate.md | 44 - docs/fr/sql-reference/table-functions/hdfs.md | 104 - .../fr/sql-reference/table-functions/index.md | 38 - .../fr/sql-reference/table-functions/input.md | 47 - docs/fr/sql-reference/table-functions/jdbc.md | 29 - .../fr/sql-reference/table-functions/merge.md | 14 - .../fr/sql-reference/table-functions/mysql.md | 86 - .../sql-reference/table-functions/numbers.md | 30 - docs/fr/sql-reference/table-functions/odbc.md | 108 - .../sql-reference/table-functions/remote.md | 85 - docs/fr/sql-reference/table-functions/url.md | 26 - docs/fr/whats-new/changelog/2017.md | 1 - docs/fr/whats-new/changelog/2018.md | 1 - docs/fr/whats-new/changelog/2019.md | 1 - docs/fr/whats-new/changelog/index.md | 1 - docs/fr/whats-new/index.md | 8 - docs/fr/whats-new/roadmap.md | 19 - docs/fr/whats-new/security-changelog.md | 76 - docs/tools/build.py | 6 +- docs/tools/make_links.sh | 2 +- website/locale/es/LC_MESSAGES/messages.mo | Bin 431 -> 0 bytes website/locale/es/LC_MESSAGES/messages.po | 326 --- website/locale/fr/LC_MESSAGES/messages.mo | Bin 430 -> 0 bytes website/locale/fr/LC_MESSAGES/messages.po | 326 --- website/sitemap-index.xml | 9 - 496 files changed, 2 insertions(+), 75541 deletions(-) delete mode 100644 docs/es/commercial/cloud.md delete mode 100644 docs/es/commercial/index.md delete mode 100644 docs/es/commercial/support.md delete mode 100644 docs/es/development/architecture.md delete mode 100644 docs/es/development/browse-code.md delete mode 100644 docs/es/development/build-cross-arm.md delete mode 100644 docs/es/development/build-cross-osx.md delete mode 100644 docs/es/development/build-osx.md delete mode 100644 docs/es/development/build.md delete mode 100644 docs/es/development/contrib.md delete mode 100644 docs/es/development/developer-instruction.md delete mode 100644 docs/es/development/index.md delete mode 100644 docs/es/development/style.md delete mode 120000 docs/es/development/tests.md delete mode 100644 docs/es/engines/database-engines/atomic.md delete mode 100644 docs/es/engines/database-engines/index.md delete mode 100644 docs/es/engines/database-engines/lazy.md delete mode 100644 docs/es/engines/database-engines/mysql.md delete mode 100644 docs/es/engines/index.md delete mode 100644 docs/es/engines/table-engines/index.md delete mode 100644 docs/es/engines/table-engines/integrations/hdfs.md delete mode 100644 docs/es/engines/table-engines/integrations/index.md delete mode 100644 docs/es/engines/table-engines/integrations/jdbc.md delete mode 100644 docs/es/engines/table-engines/integrations/kafka.md delete mode 100644 docs/es/engines/table-engines/integrations/mysql.md delete mode 100644 docs/es/engines/table-engines/integrations/odbc.md delete mode 100644 docs/es/engines/table-engines/log-family/index.md delete mode 100644 docs/es/engines/table-engines/log-family/log.md delete mode 100644 docs/es/engines/table-engines/log-family/stripelog.md delete mode 100644 docs/es/engines/table-engines/log-family/tinylog.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/aggregatingmergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/collapsingmergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/graphitemergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/index.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/mergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/replacingmergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/replication.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/summingmergetree.md delete mode 100644 docs/es/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md delete mode 100644 docs/es/engines/table-engines/special/buffer.md delete mode 100644 docs/es/engines/table-engines/special/dictionary.md delete mode 100644 docs/es/engines/table-engines/special/distributed.md delete mode 100644 docs/es/engines/table-engines/special/external-data.md delete mode 100644 docs/es/engines/table-engines/special/file.md delete mode 100644 docs/es/engines/table-engines/special/generate.md delete mode 100644 docs/es/engines/table-engines/special/index.md delete mode 100644 docs/es/engines/table-engines/special/join.md delete mode 100644 docs/es/engines/table-engines/special/materializedview.md delete mode 100644 docs/es/engines/table-engines/special/memory.md delete mode 100644 docs/es/engines/table-engines/special/merge.md delete mode 100644 docs/es/engines/table-engines/special/null.md delete mode 100644 docs/es/engines/table-engines/special/set.md delete mode 100644 docs/es/engines/table-engines/special/url.md delete mode 100644 docs/es/engines/table-engines/special/view.md delete mode 100644 docs/es/faq/general.md delete mode 100644 docs/es/faq/index.md delete mode 100644 docs/es/getting-started/example-datasets/amplab-benchmark.md delete mode 100644 docs/es/getting-started/example-datasets/criteo.md delete mode 100644 docs/es/getting-started/example-datasets/index.md delete mode 100644 docs/es/getting-started/example-datasets/metrica.md delete mode 100644 docs/es/getting-started/example-datasets/nyc-taxi.md delete mode 100644 docs/es/getting-started/example-datasets/ontime.md delete mode 100644 docs/es/getting-started/example-datasets/star-schema.md delete mode 100644 docs/es/getting-started/example-datasets/wikistat.md delete mode 100644 docs/es/getting-started/index.md delete mode 100644 docs/es/getting-started/install.md delete mode 100644 docs/es/getting-started/playground.md delete mode 100644 docs/es/getting-started/tutorial.md delete mode 100644 docs/es/guides/apply-catboost-model.md delete mode 100644 docs/es/guides/index.md delete mode 100644 docs/es/images/column-oriented.gif delete mode 100644 docs/es/images/logo.svg delete mode 100644 docs/es/images/row-oriented.gif delete mode 100644 docs/es/index.md delete mode 100644 docs/es/interfaces/cli.md delete mode 100644 docs/es/interfaces/cpp.md delete mode 100644 docs/es/interfaces/formats.md delete mode 100644 docs/es/interfaces/http.md delete mode 100644 docs/es/interfaces/index.md delete mode 100644 docs/es/interfaces/jdbc.md delete mode 100644 docs/es/interfaces/mysql.md delete mode 100644 docs/es/interfaces/odbc.md delete mode 100644 docs/es/interfaces/tcp.md delete mode 100644 docs/es/interfaces/third-party/client-libraries.md delete mode 100644 docs/es/interfaces/third-party/gui.md delete mode 100644 docs/es/interfaces/third-party/index.md delete mode 100644 docs/es/interfaces/third-party/integrations.md delete mode 100644 docs/es/interfaces/third-party/proxy.md delete mode 100644 docs/es/introduction/adopters.md delete mode 100644 docs/es/introduction/distinctive-features.md delete mode 100644 docs/es/introduction/history.md delete mode 100644 docs/es/introduction/index.md delete mode 100644 docs/es/introduction/performance.md delete mode 100644 docs/es/operations/access-rights.md delete mode 100644 docs/es/operations/backup.md delete mode 100644 docs/es/operations/configuration-files.md delete mode 100644 docs/es/operations/index.md delete mode 100644 docs/es/operations/monitoring.md delete mode 100644 docs/es/operations/optimizing-performance/index.md delete mode 100644 docs/es/operations/optimizing-performance/sampling-query-profiler.md delete mode 100644 docs/es/operations/performance-test.md delete mode 100644 docs/es/operations/quotas.md delete mode 100644 docs/es/operations/requirements.md delete mode 100644 docs/es/operations/server-configuration-parameters/index.md delete mode 100644 docs/es/operations/server-configuration-parameters/settings.md delete mode 100644 docs/es/operations/settings/constraints-on-settings.md delete mode 100644 docs/es/operations/settings/index.md delete mode 100644 docs/es/operations/settings/permissions-for-queries.md delete mode 100644 docs/es/operations/settings/query-complexity.md delete mode 100644 docs/es/operations/settings/settings-profiles.md delete mode 100644 docs/es/operations/settings/settings-users.md delete mode 100644 docs/es/operations/settings/settings.md delete mode 100644 docs/es/operations/system-tables.md delete mode 100644 docs/es/operations/tips.md delete mode 100644 docs/es/operations/troubleshooting.md delete mode 100644 docs/es/operations/update.md delete mode 100644 docs/es/operations/utilities/clickhouse-benchmark.md delete mode 100644 docs/es/operations/utilities/clickhouse-copier.md delete mode 100644 docs/es/operations/utilities/clickhouse-local.md delete mode 100644 docs/es/operations/utilities/index.md delete mode 100644 docs/es/roadmap.md delete mode 100644 docs/es/sql-reference/aggregate-functions/combinators.md delete mode 100644 docs/es/sql-reference/aggregate-functions/index.md delete mode 100644 docs/es/sql-reference/aggregate-functions/parametric-functions.md delete mode 100644 docs/es/sql-reference/aggregate-functions/reference.md delete mode 100644 docs/es/sql-reference/ansi.md delete mode 100644 docs/es/sql-reference/data-types/aggregatefunction.md delete mode 100644 docs/es/sql-reference/data-types/array.md delete mode 100644 docs/es/sql-reference/data-types/boolean.md delete mode 100644 docs/es/sql-reference/data-types/date.md delete mode 100644 docs/es/sql-reference/data-types/datetime.md delete mode 100644 docs/es/sql-reference/data-types/datetime64.md delete mode 100644 docs/es/sql-reference/data-types/decimal.md delete mode 100644 docs/es/sql-reference/data-types/domains/index.md delete mode 100644 docs/es/sql-reference/data-types/domains/ipv4.md delete mode 100644 docs/es/sql-reference/data-types/domains/ipv6.md delete mode 100644 docs/es/sql-reference/data-types/enum.md delete mode 100644 docs/es/sql-reference/data-types/fixedstring.md delete mode 100644 docs/es/sql-reference/data-types/float.md delete mode 100644 docs/es/sql-reference/data-types/index.md delete mode 100644 docs/es/sql-reference/data-types/int-uint.md delete mode 100644 docs/es/sql-reference/data-types/nested-data-structures/index.md delete mode 100644 docs/es/sql-reference/data-types/nested-data-structures/nested.md delete mode 100644 docs/es/sql-reference/data-types/nullable.md delete mode 100644 docs/es/sql-reference/data-types/simpleaggregatefunction.md delete mode 100644 docs/es/sql-reference/data-types/special-data-types/expression.md delete mode 100644 docs/es/sql-reference/data-types/special-data-types/index.md delete mode 100644 docs/es/sql-reference/data-types/special-data-types/interval.md delete mode 100644 docs/es/sql-reference/data-types/special-data-types/nothing.md delete mode 100644 docs/es/sql-reference/data-types/special-data-types/set.md delete mode 100644 docs/es/sql-reference/data-types/string.md delete mode 100644 docs/es/sql-reference/data-types/tuple.md delete mode 100644 docs/es/sql-reference/data-types/uuid.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/external-dicts.md delete mode 100644 docs/es/sql-reference/dictionaries/external-dictionaries/index.md delete mode 100644 docs/es/sql-reference/dictionaries/index.md delete mode 100644 docs/es/sql-reference/dictionaries/internal-dicts.md delete mode 100644 docs/es/sql-reference/functions/arithmetic-functions.md delete mode 100644 docs/es/sql-reference/functions/array-functions.md delete mode 100644 docs/es/sql-reference/functions/array-join.md delete mode 100644 docs/es/sql-reference/functions/bit-functions.md delete mode 100644 docs/es/sql-reference/functions/bitmap-functions.md delete mode 100644 docs/es/sql-reference/functions/comparison-functions.md delete mode 100644 docs/es/sql-reference/functions/conditional-functions.md delete mode 100644 docs/es/sql-reference/functions/date-time-functions.md delete mode 100644 docs/es/sql-reference/functions/encoding-functions.md delete mode 100644 docs/es/sql-reference/functions/ext-dict-functions.md delete mode 100644 docs/es/sql-reference/functions/functions-for-nulls.md delete mode 100644 docs/es/sql-reference/functions/geo.md delete mode 100644 docs/es/sql-reference/functions/hash-functions.md delete mode 100644 docs/es/sql-reference/functions/higher-order-functions.md delete mode 100644 docs/es/sql-reference/functions/in-functions.md delete mode 100644 docs/es/sql-reference/functions/index.md delete mode 100644 docs/es/sql-reference/functions/introspection.md delete mode 100644 docs/es/sql-reference/functions/ip-address-functions.md delete mode 100644 docs/es/sql-reference/functions/json-functions.md delete mode 100644 docs/es/sql-reference/functions/logical-functions.md delete mode 100644 docs/es/sql-reference/functions/machine-learning-functions.md delete mode 100644 docs/es/sql-reference/functions/math-functions.md delete mode 100644 docs/es/sql-reference/functions/other-functions.md delete mode 100644 docs/es/sql-reference/functions/random-functions.md delete mode 100644 docs/es/sql-reference/functions/rounding-functions.md delete mode 100644 docs/es/sql-reference/functions/splitting-merging-functions.md delete mode 100644 docs/es/sql-reference/functions/string-functions.md delete mode 100644 docs/es/sql-reference/functions/string-replace-functions.md delete mode 100644 docs/es/sql-reference/functions/string-search-functions.md delete mode 100644 docs/es/sql-reference/functions/type-conversion-functions.md delete mode 100644 docs/es/sql-reference/functions/url-functions.md delete mode 100644 docs/es/sql-reference/functions/uuid-functions.md delete mode 100644 docs/es/sql-reference/functions/ym-dict-functions.md delete mode 100644 docs/es/sql-reference/index.md delete mode 100644 docs/es/sql-reference/operators/in.md delete mode 100644 docs/es/sql-reference/operators/index.md delete mode 100644 docs/es/sql-reference/statements/alter.md delete mode 100644 docs/es/sql-reference/statements/create.md delete mode 100644 docs/es/sql-reference/statements/grant.md delete mode 100644 docs/es/sql-reference/statements/index.md delete mode 100644 docs/es/sql-reference/statements/insert-into.md delete mode 100644 docs/es/sql-reference/statements/misc.md delete mode 100644 docs/es/sql-reference/statements/revoke.md delete mode 100644 docs/es/sql-reference/statements/select/array-join.md delete mode 100644 docs/es/sql-reference/statements/select/distinct.md delete mode 100644 docs/es/sql-reference/statements/select/format.md delete mode 100644 docs/es/sql-reference/statements/select/from.md delete mode 100644 docs/es/sql-reference/statements/select/group-by.md delete mode 100644 docs/es/sql-reference/statements/select/having.md delete mode 100644 docs/es/sql-reference/statements/select/index.md delete mode 100644 docs/es/sql-reference/statements/select/into-outfile.md delete mode 100644 docs/es/sql-reference/statements/select/join.md delete mode 100644 docs/es/sql-reference/statements/select/limit-by.md delete mode 100644 docs/es/sql-reference/statements/select/limit.md delete mode 100644 docs/es/sql-reference/statements/select/order-by.md delete mode 100644 docs/es/sql-reference/statements/select/prewhere.md delete mode 100644 docs/es/sql-reference/statements/select/sample.md delete mode 100644 docs/es/sql-reference/statements/select/union.md delete mode 100644 docs/es/sql-reference/statements/select/where.md delete mode 100644 docs/es/sql-reference/statements/select/with.md delete mode 100644 docs/es/sql-reference/statements/show.md delete mode 100644 docs/es/sql-reference/statements/system.md delete mode 100644 docs/es/sql-reference/syntax.md delete mode 100644 docs/es/sql-reference/table-functions/file.md delete mode 100644 docs/es/sql-reference/table-functions/generate.md delete mode 100644 docs/es/sql-reference/table-functions/hdfs.md delete mode 100644 docs/es/sql-reference/table-functions/index.md delete mode 100644 docs/es/sql-reference/table-functions/input.md delete mode 100644 docs/es/sql-reference/table-functions/jdbc.md delete mode 100644 docs/es/sql-reference/table-functions/merge.md delete mode 100644 docs/es/sql-reference/table-functions/mysql.md delete mode 100644 docs/es/sql-reference/table-functions/numbers.md delete mode 100644 docs/es/sql-reference/table-functions/odbc.md delete mode 100644 docs/es/sql-reference/table-functions/remote.md delete mode 100644 docs/es/sql-reference/table-functions/url.md delete mode 120000 docs/es/whats-new/changelog/2017.md delete mode 120000 docs/es/whats-new/changelog/2018.md delete mode 120000 docs/es/whats-new/changelog/2019.md delete mode 120000 docs/es/whats-new/changelog/index.md delete mode 100644 docs/es/whats-new/index.md delete mode 100644 docs/es/whats-new/roadmap.md delete mode 100644 docs/es/whats-new/security-changelog.md delete mode 100644 docs/fr/commercial/cloud.md delete mode 100644 docs/fr/commercial/index.md delete mode 100644 docs/fr/commercial/support.md delete mode 100644 docs/fr/development/architecture.md delete mode 100644 docs/fr/development/browse-code.md delete mode 100644 docs/fr/development/build-cross-arm.md delete mode 100644 docs/fr/development/build-cross-osx.md delete mode 100644 docs/fr/development/build-osx.md delete mode 100644 docs/fr/development/build.md delete mode 100644 docs/fr/development/contrib.md delete mode 100644 docs/fr/development/developer-instruction.md delete mode 100644 docs/fr/development/index.md delete mode 100644 docs/fr/development/style.md delete mode 120000 docs/fr/development/tests.md delete mode 100644 docs/fr/engines/database-engines/atomic.md delete mode 100644 docs/fr/engines/database-engines/index.md delete mode 100644 docs/fr/engines/database-engines/lazy.md delete mode 100644 docs/fr/engines/database-engines/mysql.md delete mode 100644 docs/fr/engines/index.md delete mode 100644 docs/fr/engines/table-engines/index.md delete mode 100644 docs/fr/engines/table-engines/integrations/hdfs.md delete mode 100644 docs/fr/engines/table-engines/integrations/index.md delete mode 100644 docs/fr/engines/table-engines/integrations/jdbc.md delete mode 100644 docs/fr/engines/table-engines/integrations/kafka.md delete mode 100644 docs/fr/engines/table-engines/integrations/mysql.md delete mode 100644 docs/fr/engines/table-engines/integrations/odbc.md delete mode 100644 docs/fr/engines/table-engines/log-family/index.md delete mode 100644 docs/fr/engines/table-engines/log-family/log.md delete mode 100644 docs/fr/engines/table-engines/log-family/stripelog.md delete mode 100644 docs/fr/engines/table-engines/log-family/tinylog.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/aggregatingmergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/collapsingmergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/custom-partitioning-key.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/graphitemergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/index.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/mergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/replacingmergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/replication.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/summingmergetree.md delete mode 100644 docs/fr/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md delete mode 100644 docs/fr/engines/table-engines/special/buffer.md delete mode 100644 docs/fr/engines/table-engines/special/dictionary.md delete mode 100644 docs/fr/engines/table-engines/special/distributed.md delete mode 100644 docs/fr/engines/table-engines/special/external-data.md delete mode 100644 docs/fr/engines/table-engines/special/file.md delete mode 100644 docs/fr/engines/table-engines/special/generate.md delete mode 100644 docs/fr/engines/table-engines/special/index.md delete mode 100644 docs/fr/engines/table-engines/special/join.md delete mode 100644 docs/fr/engines/table-engines/special/materializedview.md delete mode 100644 docs/fr/engines/table-engines/special/memory.md delete mode 100644 docs/fr/engines/table-engines/special/merge.md delete mode 100644 docs/fr/engines/table-engines/special/null.md delete mode 100644 docs/fr/engines/table-engines/special/set.md delete mode 100644 docs/fr/engines/table-engines/special/url.md delete mode 100644 docs/fr/engines/table-engines/special/view.md delete mode 100644 docs/fr/faq/general.md delete mode 100644 docs/fr/faq/index.md delete mode 100644 docs/fr/getting-started/example-datasets/amplab-benchmark.md delete mode 100644 docs/fr/getting-started/example-datasets/criteo.md delete mode 100644 docs/fr/getting-started/example-datasets/index.md delete mode 100644 docs/fr/getting-started/example-datasets/metrica.md delete mode 100644 docs/fr/getting-started/example-datasets/nyc-taxi.md delete mode 100644 docs/fr/getting-started/example-datasets/ontime.md delete mode 100644 docs/fr/getting-started/example-datasets/star-schema.md delete mode 100644 docs/fr/getting-started/example-datasets/wikistat.md delete mode 100644 docs/fr/getting-started/index.md delete mode 100644 docs/fr/getting-started/install.md delete mode 100644 docs/fr/getting-started/playground.md delete mode 100644 docs/fr/getting-started/tutorial.md delete mode 100644 docs/fr/guides/apply-catboost-model.md delete mode 100644 docs/fr/guides/index.md delete mode 120000 docs/fr/images delete mode 100644 docs/fr/index.md delete mode 100644 docs/fr/interfaces/cli.md delete mode 100644 docs/fr/interfaces/cpp.md delete mode 100644 docs/fr/interfaces/formats.md delete mode 100644 docs/fr/interfaces/http.md delete mode 100644 docs/fr/interfaces/index.md delete mode 100644 docs/fr/interfaces/jdbc.md delete mode 100644 docs/fr/interfaces/mysql.md delete mode 100644 docs/fr/interfaces/odbc.md delete mode 100644 docs/fr/interfaces/tcp.md delete mode 100644 docs/fr/interfaces/third-party/client-libraries.md delete mode 100644 docs/fr/interfaces/third-party/gui.md delete mode 100644 docs/fr/interfaces/third-party/index.md delete mode 100644 docs/fr/interfaces/third-party/integrations.md delete mode 100644 docs/fr/interfaces/third-party/proxy.md delete mode 100644 docs/fr/introduction/adopters.md delete mode 100644 docs/fr/introduction/distinctive-features.md delete mode 100644 docs/fr/introduction/history.md delete mode 100644 docs/fr/introduction/index.md delete mode 100644 docs/fr/introduction/performance.md delete mode 100644 docs/fr/operations/access-rights.md delete mode 100644 docs/fr/operations/backup.md delete mode 100644 docs/fr/operations/configuration-files.md delete mode 100644 docs/fr/operations/index.md delete mode 100644 docs/fr/operations/monitoring.md delete mode 100644 docs/fr/operations/optimizing-performance/index.md delete mode 100644 docs/fr/operations/optimizing-performance/sampling-query-profiler.md delete mode 100644 docs/fr/operations/performance-test.md delete mode 100644 docs/fr/operations/quotas.md delete mode 100644 docs/fr/operations/requirements.md delete mode 100644 docs/fr/operations/server-configuration-parameters/index.md delete mode 100644 docs/fr/operations/server-configuration-parameters/settings.md delete mode 100644 docs/fr/operations/settings/constraints-on-settings.md delete mode 100644 docs/fr/operations/settings/index.md delete mode 100644 docs/fr/operations/settings/permissions-for-queries.md delete mode 100644 docs/fr/operations/settings/query-complexity.md delete mode 100644 docs/fr/operations/settings/settings-profiles.md delete mode 100644 docs/fr/operations/settings/settings-users.md delete mode 100644 docs/fr/operations/settings/settings.md delete mode 100644 docs/fr/operations/system-tables.md delete mode 100644 docs/fr/operations/tips.md delete mode 100644 docs/fr/operations/troubleshooting.md delete mode 100644 docs/fr/operations/update.md delete mode 100644 docs/fr/operations/utilities/clickhouse-benchmark.md delete mode 100644 docs/fr/operations/utilities/clickhouse-copier.md delete mode 100644 docs/fr/operations/utilities/clickhouse-local.md delete mode 100644 docs/fr/operations/utilities/index.md delete mode 100644 docs/fr/sql-reference/aggregate-functions/combinators.md delete mode 100644 docs/fr/sql-reference/aggregate-functions/index.md delete mode 100644 docs/fr/sql-reference/aggregate-functions/parametric-functions.md delete mode 100644 docs/fr/sql-reference/aggregate-functions/reference.md delete mode 100644 docs/fr/sql-reference/ansi.md delete mode 100644 docs/fr/sql-reference/data-types/aggregatefunction.md delete mode 100644 docs/fr/sql-reference/data-types/array.md delete mode 100644 docs/fr/sql-reference/data-types/boolean.md delete mode 100644 docs/fr/sql-reference/data-types/date.md delete mode 100644 docs/fr/sql-reference/data-types/datetime.md delete mode 100644 docs/fr/sql-reference/data-types/datetime64.md delete mode 100644 docs/fr/sql-reference/data-types/decimal.md delete mode 100644 docs/fr/sql-reference/data-types/domains/index.md delete mode 100644 docs/fr/sql-reference/data-types/domains/ipv4.md delete mode 100644 docs/fr/sql-reference/data-types/domains/ipv6.md delete mode 100644 docs/fr/sql-reference/data-types/enum.md delete mode 100644 docs/fr/sql-reference/data-types/fixedstring.md delete mode 100644 docs/fr/sql-reference/data-types/float.md delete mode 100644 docs/fr/sql-reference/data-types/index.md delete mode 100644 docs/fr/sql-reference/data-types/int-uint.md delete mode 100644 docs/fr/sql-reference/data-types/nested-data-structures/index.md delete mode 100644 docs/fr/sql-reference/data-types/nested-data-structures/nested.md delete mode 100644 docs/fr/sql-reference/data-types/nullable.md delete mode 100644 docs/fr/sql-reference/data-types/simpleaggregatefunction.md delete mode 100644 docs/fr/sql-reference/data-types/special-data-types/expression.md delete mode 100644 docs/fr/sql-reference/data-types/special-data-types/index.md delete mode 100644 docs/fr/sql-reference/data-types/special-data-types/interval.md delete mode 100644 docs/fr/sql-reference/data-types/special-data-types/nothing.md delete mode 100644 docs/fr/sql-reference/data-types/special-data-types/set.md delete mode 100644 docs/fr/sql-reference/data-types/string.md delete mode 100644 docs/fr/sql-reference/data-types/tuple.md delete mode 100644 docs/fr/sql-reference/data-types/uuid.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md delete mode 100644 docs/fr/sql-reference/dictionaries/external-dictionaries/index.md delete mode 100644 docs/fr/sql-reference/dictionaries/index.md delete mode 100644 docs/fr/sql-reference/dictionaries/internal-dicts.md delete mode 100644 docs/fr/sql-reference/functions/arithmetic-functions.md delete mode 100644 docs/fr/sql-reference/functions/array-functions.md delete mode 100644 docs/fr/sql-reference/functions/array-join.md delete mode 100644 docs/fr/sql-reference/functions/bit-functions.md delete mode 100644 docs/fr/sql-reference/functions/bitmap-functions.md delete mode 100644 docs/fr/sql-reference/functions/comparison-functions.md delete mode 100644 docs/fr/sql-reference/functions/conditional-functions.md delete mode 100644 docs/fr/sql-reference/functions/date-time-functions.md delete mode 100644 docs/fr/sql-reference/functions/encoding-functions.md delete mode 100644 docs/fr/sql-reference/functions/ext-dict-functions.md delete mode 100644 docs/fr/sql-reference/functions/functions-for-nulls.md delete mode 100644 docs/fr/sql-reference/functions/geo.md delete mode 100644 docs/fr/sql-reference/functions/hash-functions.md delete mode 100644 docs/fr/sql-reference/functions/higher-order-functions.md delete mode 100644 docs/fr/sql-reference/functions/in-functions.md delete mode 100644 docs/fr/sql-reference/functions/index.md delete mode 100644 docs/fr/sql-reference/functions/introspection.md delete mode 100644 docs/fr/sql-reference/functions/ip-address-functions.md delete mode 100644 docs/fr/sql-reference/functions/json-functions.md delete mode 100644 docs/fr/sql-reference/functions/logical-functions.md delete mode 100644 docs/fr/sql-reference/functions/machine-learning-functions.md delete mode 100644 docs/fr/sql-reference/functions/math-functions.md delete mode 100644 docs/fr/sql-reference/functions/other-functions.md delete mode 100644 docs/fr/sql-reference/functions/random-functions.md delete mode 100644 docs/fr/sql-reference/functions/rounding-functions.md delete mode 100644 docs/fr/sql-reference/functions/splitting-merging-functions.md delete mode 100644 docs/fr/sql-reference/functions/string-functions.md delete mode 100644 docs/fr/sql-reference/functions/string-replace-functions.md delete mode 100644 docs/fr/sql-reference/functions/string-search-functions.md delete mode 100644 docs/fr/sql-reference/functions/type-conversion-functions.md delete mode 100644 docs/fr/sql-reference/functions/url-functions.md delete mode 100644 docs/fr/sql-reference/functions/uuid-functions.md delete mode 100644 docs/fr/sql-reference/functions/ym-dict-functions.md delete mode 100644 docs/fr/sql-reference/index.md delete mode 100644 docs/fr/sql-reference/operators/in.md delete mode 100644 docs/fr/sql-reference/operators/index.md delete mode 100644 docs/fr/sql-reference/statements/alter.md delete mode 100644 docs/fr/sql-reference/statements/create.md delete mode 100644 docs/fr/sql-reference/statements/grant.md delete mode 100644 docs/fr/sql-reference/statements/index.md delete mode 100644 docs/fr/sql-reference/statements/insert-into.md delete mode 100644 docs/fr/sql-reference/statements/misc.md delete mode 100644 docs/fr/sql-reference/statements/revoke.md delete mode 100644 docs/fr/sql-reference/statements/select/array-join.md delete mode 100644 docs/fr/sql-reference/statements/select/distinct.md delete mode 100644 docs/fr/sql-reference/statements/select/format.md delete mode 100644 docs/fr/sql-reference/statements/select/from.md delete mode 100644 docs/fr/sql-reference/statements/select/group-by.md delete mode 100644 docs/fr/sql-reference/statements/select/having.md delete mode 100644 docs/fr/sql-reference/statements/select/index.md delete mode 100644 docs/fr/sql-reference/statements/select/into-outfile.md delete mode 100644 docs/fr/sql-reference/statements/select/join.md delete mode 100644 docs/fr/sql-reference/statements/select/limit-by.md delete mode 100644 docs/fr/sql-reference/statements/select/limit.md delete mode 100644 docs/fr/sql-reference/statements/select/order-by.md delete mode 100644 docs/fr/sql-reference/statements/select/prewhere.md delete mode 100644 docs/fr/sql-reference/statements/select/sample.md delete mode 100644 docs/fr/sql-reference/statements/select/union.md delete mode 100644 docs/fr/sql-reference/statements/select/where.md delete mode 100644 docs/fr/sql-reference/statements/select/with.md delete mode 100644 docs/fr/sql-reference/statements/show.md delete mode 100644 docs/fr/sql-reference/statements/system.md delete mode 100644 docs/fr/sql-reference/syntax.md delete mode 100644 docs/fr/sql-reference/table-functions/file.md delete mode 100644 docs/fr/sql-reference/table-functions/generate.md delete mode 100644 docs/fr/sql-reference/table-functions/hdfs.md delete mode 100644 docs/fr/sql-reference/table-functions/index.md delete mode 100644 docs/fr/sql-reference/table-functions/input.md delete mode 100644 docs/fr/sql-reference/table-functions/jdbc.md delete mode 100644 docs/fr/sql-reference/table-functions/merge.md delete mode 100644 docs/fr/sql-reference/table-functions/mysql.md delete mode 100644 docs/fr/sql-reference/table-functions/numbers.md delete mode 100644 docs/fr/sql-reference/table-functions/odbc.md delete mode 100644 docs/fr/sql-reference/table-functions/remote.md delete mode 100644 docs/fr/sql-reference/table-functions/url.md delete mode 120000 docs/fr/whats-new/changelog/2017.md delete mode 120000 docs/fr/whats-new/changelog/2018.md delete mode 120000 docs/fr/whats-new/changelog/2019.md delete mode 120000 docs/fr/whats-new/changelog/index.md delete mode 100644 docs/fr/whats-new/index.md delete mode 100644 docs/fr/whats-new/roadmap.md delete mode 100644 docs/fr/whats-new/security-changelog.md delete mode 100644 website/locale/es/LC_MESSAGES/messages.mo delete mode 100644 website/locale/es/LC_MESSAGES/messages.po delete mode 100644 website/locale/fr/LC_MESSAGES/messages.mo delete mode 100644 website/locale/fr/LC_MESSAGES/messages.po diff --git a/docs/es/commercial/cloud.md b/docs/es/commercial/cloud.md deleted file mode 100644 index bc593a82ad7..00000000000 --- a/docs/es/commercial/cloud.md +++ /dev/null @@ -1,23 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 1 -toc_title: Nube ---- - -# Proveedores de servicios en la nube de ClickHouse {#clickhouse-cloud-service-providers} - -!!! info "INFO" - Si ha lanzado una nube pública con el servicio ClickHouse administrado, no dude en [abrir una solicitud de extracción](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/cloud.md) añadiéndolo a la siguiente lista. - -## Nube de Yandex {#yandex-cloud} - -[Servicio administrado de Yandex para ClickHouse](https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link3) proporciona las siguientes características clave: - -- Servicio ZooKeeper totalmente gestionado para [Replicación de ClickHouse](../engines/table-engines/mergetree-family/replication.md) -- Múltiples opciones de tipo de almacenamiento -- Réplicas en diferentes zonas de disponibilidad -- Cifrado y aislamiento -- Mantenimiento automatizado - -{## [Artículo Original](https://clickhouse.tech/docs/en/commercial/cloud/) ##} diff --git a/docs/es/commercial/index.md b/docs/es/commercial/index.md deleted file mode 100644 index b367631ae1c..00000000000 --- a/docs/es/commercial/index.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Comercial -toc_priority: 70 -toc_title: Comercial ---- - - diff --git a/docs/es/commercial/support.md b/docs/es/commercial/support.md deleted file mode 100644 index a817d90dcb5..00000000000 --- a/docs/es/commercial/support.md +++ /dev/null @@ -1,23 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 3 -toc_title: Apoyo ---- - -# Proveedores de servicios de soporte comercial ClickHouse {#clickhouse-commercial-support-service-providers} - -!!! info "INFO" - Si ha lanzado un servicio de soporte comercial ClickHouse, no dude en [abrir una solicitud de extracción](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) añadiéndolo a la siguiente lista. - -## Altinidad {#altinity} - -Altinity ha ofrecido soporte y servicios empresariales ClickHouse desde 2017. Los clientes de Altinity van desde empresas Fortune 100 hasta startups. Visitar [Más información](https://www.altinity.com/) para más información. - -## Mafiree {#mafiree} - -[Descripción del servicio](http://mafiree.com/clickhouse-analytics-services.php) - -## MinervaDB {#minervadb} - -[Descripción del servicio](https://minervadb.com/index.php/clickhouse-consulting-and-support-by-minervadb/) diff --git a/docs/es/development/architecture.md b/docs/es/development/architecture.md deleted file mode 100644 index 1620a58a3a0..00000000000 --- a/docs/es/development/architecture.md +++ /dev/null @@ -1,203 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 62 -toc_title: "Descripci\xF3n general de la arquitectura ClickHouse" ---- - -# Descripción general de la arquitectura ClickHouse {#overview-of-clickhouse-architecture} - -ClickHouse es un verdadero DBMS orientado a columnas. Los datos se almacenan por columnas y durante la ejecución de matrices (vectores o fragmentos de columnas). Siempre que sea posible, las operaciones se envían en matrices, en lugar de en valores individuales. Se llama “vectorized query execution,” y ayuda a reducir el costo del procesamiento de datos real. - -> Esta idea no es nada nuevo. Se remonta a la `APL` lenguaje de programación y sus descendientes: `A +`, `J`, `K`, y `Q`. La programación de matrices se utiliza en el procesamiento de datos científicos. Tampoco es esta idea algo nuevo en las bases de datos relacionales: por ejemplo, se usa en el `Vectorwise` sistema. - -Existen dos enfoques diferentes para acelerar el procesamiento de consultas: la ejecución de consultas vectorizadas y la generación de código en tiempo de ejecución. Este último elimina toda la indirección y el despacho dinámico. Ninguno de estos enfoques es estrictamente mejor que el otro. La generación de código de tiempo de ejecución puede ser mejor cuando fusiona muchas operaciones, utilizando así las unidades de ejecución de la CPU y la canalización. La ejecución de consultas vectorizadas puede ser menos práctica porque implica vectores temporales que deben escribirse en la memoria caché y leerse. Si los datos temporales no caben en la memoria caché L2, esto se convierte en un problema. Pero la ejecución de consultas vectorizadas utiliza más fácilmente las capacidades SIMD de la CPU. Un [documento de investigación](http://15721.courses.cs.cmu.edu/spring2016/papers/p5-sompolski.pdf) escrito por nuestros amigos muestra que es mejor combinar ambos enfoques. ClickHouse utiliza la ejecución de consultas vectorizadas y tiene un soporte inicial limitado para la generación de código en tiempo de ejecución. - -## Columna {#columns} - -`IColumn` interfaz se utiliza para representar columnas en la memoria (en realidad, fragmentos de columnas). Esta interfaz proporciona métodos auxiliares para la implementación de varios operadores relacionales. Casi todas las operaciones son inmutables: no modifican la columna original, sino que crean una nueva modificada. Por ejemplo, el `IColumn :: filter` método acepta una máscara de bytes de filtro. Se utiliza para el `WHERE` y `HAVING` operadores relacionales. Ejemplos adicionales: el `IColumn :: permute` para apoyar `ORDER BY`, el `IColumn :: cut` para apoyar `LIMIT`. - -Diversos `IColumn` aplicación (`ColumnUInt8`, `ColumnString`, y así sucesivamente) son responsables del diseño de memoria de las columnas. El diseño de memoria suele ser una matriz contigua. Para el tipo entero de columnas, es solo una matriz contigua, como `std :: vector`. Para `String` y `Array` columnas, son dos vectores: uno para todos los elementos de la matriz, colocados contiguamente, y un segundo para los desplazamientos al comienzo de cada matriz. También hay `ColumnConst` que almacena solo un valor en la memoria, pero parece una columna. - -## Campo {#field} - -Sin embargo, también es posible trabajar con valores individuales. Para representar un valor individual, el `Field` se utiliza. `Field` es sólo una unión discriminada de `UInt64`, `Int64`, `Float64`, `String` y `Array`. `IColumn` tiene el `operator[]` para obtener el valor n-ésimo como un `Field` y el `insert` método para agregar un `Field` al final de una columna. Estos métodos no son muy eficientes, ya que requieren tratar con temporal `Field` objetos que representan un valor individual. Hay métodos más eficientes, tales como `insertFrom`, `insertRangeFrom` y así sucesivamente. - -`Field` no tiene suficiente información sobre un tipo de datos específico para una tabla. Por ejemplo, `UInt8`, `UInt16`, `UInt32`, y `UInt64` todos están representados como `UInt64` en una `Field`. - -## Abstracciones con fugas {#leaky-abstractions} - -`IColumn` tiene métodos para transformaciones relacionales comunes de datos, pero no satisfacen todas las necesidades. Por ejemplo, `ColumnUInt64` no tiene un método para calcular la suma de dos columnas, y `ColumnString` no tiene un método para ejecutar una búsqueda de subcadena. Estas innumerables rutinas se implementan fuera de `IColumn`. - -Varias funciones en columnas se pueden implementar de una manera genérica, no eficiente utilizando `IColumn` para extraer `Field` valores, o de una manera especializada utilizando el conocimiento del diseño de la memoria interna de los datos en un `IColumn` aplicación. Se implementa mediante la conversión de funciones a un `IColumn` escriba y trate con la representación interna directamente. Por ejemplo, `ColumnUInt64` tiene el `getData` método que devuelve una referencia a una matriz interna, luego una rutina separada lee o llena esa matriz directamente. Tenemos “leaky abstractions” para permitir especializaciones eficientes de varias rutinas. - -## Tipos de datos {#data_types} - -`IDataType` es responsable de la serialización y deserialización: para leer y escribir fragmentos de columnas o valores individuales en formato binario o de texto. `IDataType` corresponde directamente a los tipos de datos en las tablas. Por ejemplo, hay `DataTypeUInt32`, `DataTypeDateTime`, `DataTypeString` y así sucesivamente. - -`IDataType` y `IColumn` están vagamente relacionados entre sí. Diferentes tipos de datos se pueden representar en la memoria por el mismo `IColumn` aplicación. Por ejemplo, `DataTypeUInt32` y `DataTypeDateTime` están representados por `ColumnUInt32` o `ColumnConstUInt32`. Además, el mismo tipo de datos se puede representar mediante `IColumn` aplicación. Por ejemplo, `DataTypeUInt8` puede ser representado por `ColumnUInt8` o `ColumnConstUInt8`. - -`IDataType` sólo almacena metadatos. Por ejemplo, `DataTypeUInt8` no almacena nada en absoluto (excepto vptr) y `DataTypeFixedString` tiendas solo `N` (el tamaño de las cadenas de tamaño fijo). - -`IDataType` tiene métodos auxiliares para varios formatos de datos. Los ejemplos son métodos para serializar un valor con posibles citas, para serializar un valor para JSON y para serializar un valor como parte del formato XML. No hay correspondencia directa con los formatos de datos. Por ejemplo, los diferentes formatos de datos `Pretty` y `TabSeparated` puede utilizar el mismo `serializeTextEscaped` método de ayuda de la `IDataType` interfaz. - -## Bloque {#block} - -A `Block` es un contenedor que representa un subconjunto (porción) de una tabla en la memoria. Es sólo un conjunto de triples: `(IColumn, IDataType, column name)`. Durante la ejecución de la consulta, los datos son procesados por `Block`s. Si tenemos un `Block`, tenemos datos (en el `IColumn` objeto), tenemos información sobre su tipo (en `IDataType`) que nos dice cómo lidiar con esa columna, y tenemos el nombre de la columna. Podría ser el nombre de columna original de la tabla o algún nombre artificial asignado para obtener resultados temporales de los cálculos. - -Cuando calculamos alguna función sobre columnas en un bloque, agregamos otra columna con su resultado al bloque, y no tocamos columnas para argumentos de la función porque las operaciones son inmutables. Más tarde, las columnas innecesarias se pueden eliminar del bloque, pero no se pueden modificar. Es conveniente para la eliminación de subexpresiones comunes. - -Se crean bloques para cada fragmento de datos procesado. Tenga en cuenta que para el mismo tipo de cálculo, los nombres y tipos de columna siguen siendo los mismos para diferentes bloques y solo cambian los datos de columna. Es mejor dividir los datos del bloque desde el encabezado del bloque porque los tamaños de bloque pequeños tienen una gran sobrecarga de cadenas temporales para copiar shared_ptrs y nombres de columna. - -## Bloquear flujos {#block-streams} - -Los flujos de bloques son para procesar datos. Usamos flujos de bloques para leer datos de algún lugar, realizar transformaciones de datos o escribir datos en algún lugar. `IBlockInputStream` tiene el `read` método para buscar el siguiente bloque mientras esté disponible. `IBlockOutputStream` tiene el `write` método para empujar el bloque en alguna parte. - -Los flujos son responsables de: - -1. Leer o escribir en una mesa. La tabla solo devuelve una secuencia para leer o escribir bloques. -2. Implementación de formatos de datos. Por ejemplo, si desea enviar datos a un terminal en `Pretty` formato, crea un flujo de salida de bloque donde presiona bloques y los formatea. -3. Realización de transformaciones de datos. Digamos que tienes `IBlockInputStream` y desea crear una secuencia filtrada. Usted crea `FilterBlockInputStream` e inicializarlo con su transmisión. Luego, cuando tiras de un bloque de `FilterBlockInputStream`, extrae un bloque de su flujo, lo filtra y le devuelve el bloque filtrado. Las canalizaciones de ejecución de consultas se representan de esta manera. - -Hay transformaciones más sofisticadas. Por ejemplo, cuando tiras de `AggregatingBlockInputStream`, lee todos los datos de su origen, los agrega y, a continuación, devuelve un flujo de datos agregados para usted. Otro ejemplo: `UnionBlockInputStream` acepta muchas fuentes de entrada en el constructor y también una serie de subprocesos. Lanza múltiples hilos y lee de múltiples fuentes en paralelo. - -> Las secuencias de bloques usan el “pull” enfoque para controlar el flujo: cuando extrae un bloque de la primera secuencia, en consecuencia extrae los bloques requeridos de las secuencias anidadas, y toda la tubería de ejecución funcionará. Ni “pull” ni “push” es la mejor solución, porque el flujo de control está implícito y eso limita la implementación de varias características, como la ejecución simultánea de múltiples consultas (fusionando muchas tuberías). Esta limitación podría superarse con coroutines o simplemente ejecutando hilos adicionales que se esperan el uno al otro. Podemos tener más posibilidades si hacemos explícito el flujo de control: si localizamos la lógica para pasar datos de una unidad de cálculo a otra fuera de esas unidades de cálculo. Lea esto [artículo](http://journal.stuffwithstuff.com/2013/01/13/iteration-inside-and-out/) para más pensamientos. - -Debemos tener en cuenta que la canalización de ejecución de consultas crea datos temporales en cada paso. Tratamos de mantener el tamaño del bloque lo suficientemente pequeño para que los datos temporales se ajusten a la memoria caché de la CPU. Con esa suposición, escribir y leer datos temporales es casi gratis en comparación con otros cálculos. Podríamos considerar una alternativa, que es fusionar muchas operaciones en la tubería. Podría hacer que la tubería sea lo más corta posible y eliminar gran parte de los datos temporales, lo que podría ser una ventaja, pero también tiene inconvenientes. Por ejemplo, una canalización dividida facilita la implementación de almacenamiento en caché de datos intermedios, el robo de datos intermedios de consultas similares que se ejecutan al mismo tiempo y la fusión de canalizaciones para consultas similares. - -## Formato {#formats} - -Los formatos de datos se implementan con flujos de bloques. Hay “presentational” sólo es adecuado para la salida de datos al cliente, tales como `Pretty` formato, que proporciona sólo `IBlockOutputStream`. Y hay formatos de entrada / salida, como `TabSeparated` o `JSONEachRow`. - -También hay secuencias de filas: `IRowInputStream` y `IRowOutputStream`. Permiten pull/push datos por filas individuales, no por bloques. Y solo son necesarios para simplificar la implementación de formatos orientados a filas. Envoltura `BlockInputStreamFromRowInputStream` y `BlockOutputStreamFromRowOutputStream` le permite convertir flujos orientados a filas en flujos regulares orientados a bloques. - -## I/O {#io} - -Para la entrada / salida orientada a bytes, hay `ReadBuffer` y `WriteBuffer` clases abstractas. Se usan en lugar de C ++ `iostream`s. No se preocupe: cada proyecto maduro de C ++ está usando algo más que `iostream`s por buenas razones. - -`ReadBuffer` y `WriteBuffer` son solo un búfer contiguo y un cursor apuntando a la posición en ese búfer. Las implementaciones pueden poseer o no la memoria del búfer. Hay un método virtual para llenar el búfer con los siguientes datos (para `ReadBuffer`) o para vaciar el búfer en algún lugar (para `WriteBuffer`). Los métodos virtuales rara vez se llaman. - -Implementaciones de `ReadBuffer`/`WriteBuffer` se utilizan para trabajar con archivos y descriptores de archivos y sockets de red, para implementar la compresión (`CompressedWriteBuffer` is initialized with another WriteBuffer and performs compression before writing data to it), and for other purposes – the names `ConcatReadBuffer`, `LimitReadBuffer`, y `HashingWriteBuffer` hablar por sí mismos. - -Read / WriteBuffers solo se ocupan de bytes. Hay funciones de `ReadHelpers` y `WriteHelpers` archivos de encabezado para ayudar con el formato de entrada / salida. Por ejemplo, hay ayudantes para escribir un número en formato decimal. - -Veamos qué sucede cuando quieres escribir un conjunto de resultados en `JSON` formato a stdout. Tiene un conjunto de resultados listo para ser recuperado de `IBlockInputStream`. Usted crea `WriteBufferFromFileDescriptor(STDOUT_FILENO)` para escribir bytes en stdout. Usted crea `JSONRowOutputStream`, inicializado con eso `WriteBuffer` para escribir filas en `JSON` a stdout. Usted crea `BlockOutputStreamFromRowOutputStream` encima de él, para representarlo como `IBlockOutputStream`. Entonces usted llama `copyData` para transferir datos desde `IBlockInputStream` a `IBlockOutputStream` y todo funciona. Internamente, `JSONRowOutputStream` escribirá varios delimitadores JSON y llamará al `IDataType::serializeTextJSON` con una referencia a `IColumn` y el número de fila como argumentos. Consecuentemente, `IDataType::serializeTextJSON` llamará a un método de `WriteHelpers.h`: por ejemplo, `writeText` para tipos numéricos y `writeJSONString` para `DataTypeString`. - -## Tabla {#tables} - -El `IStorage` interfaz representa tablas. Las diferentes implementaciones de esa interfaz son diferentes motores de tabla. Los ejemplos son `StorageMergeTree`, `StorageMemory` y así sucesivamente. Las instancias de estas clases son solo tablas. - -Clave `IStorage` son `read` y `write`. También hay `alter`, `rename`, `drop` y así sucesivamente. El `read` método acepta los siguientes argumentos: el conjunto de columnas para leer de una tabla, el `AST` consulta a considerar, y el número deseado de flujos para devolver. Devuelve uno o varios `IBlockInputStream` objetos e información sobre la etapa de procesamiento de datos que se completó dentro de un motor de tablas durante la ejecución de la consulta. - -En la mayoría de los casos, el método de lectura solo es responsable de leer las columnas especificadas de una tabla, no de ningún procesamiento de datos adicional. Todo el procesamiento de datos adicional es realizado por el intérprete de consultas y está fuera de la responsabilidad de `IStorage`. - -Pero hay excepciones notables: - -- La consulta AST se pasa al `read` método, y el motor de tablas puede usarlo para derivar el uso del índice y leer menos datos de una tabla. -- A veces, el motor de tablas puede procesar los datos a una etapa específica. Por ejemplo, `StorageDistributed` puede enviar una consulta a servidores remotos, pedirles que procesen datos a una etapa donde se puedan fusionar datos de diferentes servidores remotos y devolver esos datos preprocesados. El intérprete de consultas termina de procesar los datos. - -Tabla `read` método puede devolver múltiples `IBlockInputStream` objetos para permitir el procesamiento de datos en paralelo. Estos flujos de entrada de bloques múltiples pueden leer de una tabla en paralelo. A continuación, puede ajustar estas secuencias con varias transformaciones (como la evaluación de expresiones o el filtrado) que se pueden calcular de forma independiente y crear un `UnionBlockInputStream` encima de ellos, para leer desde múltiples flujos en paralelo. - -También hay `TableFunction`s. Estas son funciones que devuelven un `IStorage` objeto a utilizar en el `FROM` cláusula de una consulta. - -Para tener una idea rápida de cómo implementar su motor de tabla, vea algo simple, como `StorageMemory` o `StorageTinyLog`. - -> Como resultado de la `read` método, `IStorage` devoluciones `QueryProcessingStage` – information about what parts of the query were already calculated inside storage. - -## Analizador {#parsers} - -Un analizador de descenso recursivo escrito a mano analiza una consulta. Por ejemplo, `ParserSelectQuery` simplemente llama recursivamente a los analizadores subyacentes para varias partes de la consulta. Los analizadores crean un `AST`. El `AST` está representado por nodos, que son instancias de `IAST`. - -> Los generadores de analizadores no se utilizan por razones históricas. - -## Interprete {#interpreters} - -Los intérpretes son responsables de crear la canalización de ejecución de consultas `AST`. Hay intérpretes simples, como `InterpreterExistsQuery` y `InterpreterDropQuery` o el más sofisticado `InterpreterSelectQuery`. La canalización de ejecución de consultas es una combinación de flujos de entrada o salida de bloques. Por ejemplo, el resultado de interpretar el `SELECT` la consulta es la `IBlockInputStream` para leer el conjunto de resultados; el resultado de la consulta INSERT es el `IBlockOutputStream` para escribir datos para su inserción, y el resultado de interpretar el `INSERT SELECT` la consulta es la `IBlockInputStream` que devuelve un conjunto de resultados vacío en la primera lectura, pero que copia datos de `SELECT` a `INSERT` al mismo tiempo. - -`InterpreterSelectQuery` utilizar `ExpressionAnalyzer` y `ExpressionActions` maquinaria para el análisis de consultas y transformaciones. Aquí es donde se realizan la mayoría de las optimizaciones de consultas basadas en reglas. `ExpressionAnalyzer` es bastante complicado y debe reescribirse: se deben extraer varias transformaciones de consultas y optimizaciones para separar clases para permitir transformaciones modulares o consultas. - -## Función {#functions} - -Hay funciones ordinarias y funciones agregadas. Para las funciones agregadas, consulte la siguiente sección. - -Ordinary functions don't change the number of rows – they work as if they are processing each row independently. In fact, functions are not called for individual rows, but for `Block`de datos para implementar la ejecución de consultas vectorizadas. - -Hay algunas funciones diversas, como [BlockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), y [runningAccumulate](../sql-reference/functions/other-functions.md#function-runningaccumulate), que explotan el procesamiento de bloques y violan la independencia de las filas. - -ClickHouse tiene una tipificación fuerte, por lo que no hay conversión de tipo implícita. Si una función no admite una combinación específica de tipos, produce una excepción. Pero las funciones pueden funcionar (estar sobrecargadas) para muchas combinaciones diferentes de tipos. Por ejemplo, el `plus` función (para implementar el `+` operador) funciona para cualquier combinación de tipos numéricos: `UInt8` + `Float32`, `UInt16` + `Int8` y así sucesivamente. Además, algunas funciones variadas pueden aceptar cualquier número de argumentos, como el `concat` función. - -Implementar una función puede ser un poco inconveniente porque una función distribuye explícitamente tipos de datos compatibles y `IColumns`. Por ejemplo, el `plus` La función tiene código generado por la creación de instancias de una plantilla de C ++ para cada combinación de tipos numéricos y argumentos izquierdo y derecho constantes o no constantes. - -Es un excelente lugar para implementar la generación de código en tiempo de ejecución para evitar la hinchazón del código de plantilla. Además, permite agregar funciones fusionadas como multiplicar-agregar fusionado o hacer comparaciones múltiples en una iteración de bucle. - -Debido a la ejecución de consultas vectorizadas, las funciones no se cortocircuitan. Por ejemplo, si escribe `WHERE f(x) AND g(y)`, ambos lados se calculan, incluso para las filas, cuando `f(x)` es cero (excepto cuando `f(x)` es una expresión constante cero). Pero si la selectividad del `f(x)` la condición es alta, y el cálculo de `f(x)` es mucho más barato que `g(y)`, es mejor implementar el cálculo de paso múltiple. Primero calcularía `f(x)`, a continuación, filtrar columnas por el resultado, y luego calcular `g(y)` solo para trozos de datos más pequeños y filtrados. - -## Funciones agregadas {#aggregate-functions} - -Las funciones agregadas son funciones con estado. Acumulan valores pasados en algún estado y le permiten obtener resultados de ese estado. Se gestionan con el `IAggregateFunction` interfaz. Los estados pueden ser bastante simples (el estado para `AggregateFunctionCount` es sólo una sola `UInt64` valor) o bastante complejo (el estado de `AggregateFunctionUniqCombined` es una combinación de una matriz lineal, una tabla hash, y un `HyperLogLog` estructura de datos probabilística). - -Los Estados están asignados en `Arena` (un grupo de memoria) para tratar con múltiples estados mientras se ejecuta una alta cardinalidad `GROUP BY` consulta. Los estados pueden tener un constructor y destructor no trivial: por ejemplo, los estados de agregación complicados pueden asignar memoria adicional ellos mismos. Requiere cierta atención a la creación y destrucción de estados y a la adecuada aprobación de su orden de propiedad y destrucción. - -Los estados de agregación se pueden serializar y deserializar para pasar a través de la red durante la ejecución de consultas distribuidas o para escribirlos en el disco donde no hay suficiente RAM. Incluso se pueden almacenar en una tabla con el `DataTypeAggregateFunction` para permitir la agregación incremental de datos. - -> El formato de datos serializados para los estados de función agregados no tiene versiones en este momento. Está bien si los estados agregados solo se almacenan temporalmente. Pero tenemos el `AggregatingMergeTree` motor de tabla para la agregación incremental, y la gente ya lo está utilizando en producción. Es la razón por la que se requiere compatibilidad con versiones anteriores al cambiar el formato serializado para cualquier función agregada en el futuro. - -## Servidor {#server} - -El servidor implementa varias interfaces diferentes: - -- Una interfaz HTTP para cualquier cliente extranjero. -- Una interfaz TCP para el cliente nativo de ClickHouse y para la comunicación entre servidores durante la ejecución de consultas distribuidas. -- Una interfaz para transferir datos para la replicación. - -Internamente, es solo un servidor multiproceso primitivo sin corutinas ni fibras. Dado que el servidor no está diseñado para procesar una alta tasa de consultas simples, sino para procesar una tasa relativamente baja de consultas complejas, cada uno de ellos puede procesar una gran cantidad de datos para análisis. - -El servidor inicializa el `Context` clase con el entorno necesario para la ejecución de consultas: la lista de bases de datos disponibles, usuarios y derechos de acceso, configuración, clústeres, la lista de procesos, el registro de consultas, etc. Los intérpretes utilizan este entorno. - -Mantenemos una compatibilidad total con versiones anteriores y posteriores para el protocolo TCP del servidor: los clientes antiguos pueden hablar con servidores nuevos y los nuevos clientes pueden hablar con servidores antiguos. Pero no queremos mantenerlo eternamente, y estamos eliminando el soporte para versiones antiguas después de aproximadamente un año. - -!!! note "Nota" - Para la mayoría de las aplicaciones externas, recomendamos usar la interfaz HTTP porque es simple y fácil de usar. El protocolo TCP está más estrechamente vinculado a las estructuras de datos internas: utiliza un formato interno para pasar bloques de datos y utiliza marcos personalizados para datos comprimidos. No hemos lanzado una biblioteca C para ese protocolo porque requiere vincular la mayor parte de la base de código ClickHouse, lo cual no es práctico. - -## Ejecución de consultas distribuidas {#distributed-query-execution} - -Los servidores de una configuración de clúster son en su mayoría independientes. Puede crear un `Distributed` en uno o todos los servidores de un clúster. El `Distributed` table does not store data itself – it only provides a “view” a todas las tablas locales en varios nodos de un clúster. Cuando se SELECCIONA desde un `Distributed` tabla, reescribe esa consulta, elige nodos remotos de acuerdo con la configuración de equilibrio de carga y les envía la consulta. El `Distributed` table solicita a los servidores remotos que procesen una consulta hasta una etapa en la que se pueden fusionar resultados intermedios de diferentes servidores. Luego recibe los resultados intermedios y los fusiona. La tabla distribuida intenta distribuir tanto trabajo como sea posible a servidores remotos y no envía muchos datos intermedios a través de la red. - -Las cosas se vuelven más complicadas cuando tiene subconsultas en cláusulas IN o JOIN, y cada una de ellas usa un `Distributed` tabla. Tenemos diferentes estrategias para la ejecución de estas consultas. - -No existe un plan de consulta global para la ejecución de consultas distribuidas. Cada nodo tiene su plan de consulta local para su parte del trabajo. Solo tenemos una ejecución simple de consultas distribuidas de un solo paso: enviamos consultas para nodos remotos y luego fusionamos los resultados. Pero esto no es factible para consultas complicadas con alta cardinalidad GROUP BY o con una gran cantidad de datos temporales para JOIN. En tales casos, necesitamos “reshuffle” datos entre servidores, lo que requiere una coordinación adicional. ClickHouse no admite ese tipo de ejecución de consultas, y tenemos que trabajar en ello. - -## Árbol de fusión {#merge-tree} - -`MergeTree` es una familia de motores de almacenamiento que admite la indexación por clave principal. La clave principal puede ser una tupla arbitraria de columnas o expresiones. Datos en un `MergeTree` se almacena en “parts”. Cada parte almacena los datos en el orden de la clave principal, por lo que la tupla de la clave principal ordena los datos lexicográficamente. Todas las columnas de la tabla se almacenan en `column.bin` archivos en estas partes. Los archivos consisten en bloques comprimidos. Cada bloque suele ser de 64 KB a 1 MB de datos sin comprimir, dependiendo del tamaño del valor promedio. Los bloques constan de valores de columna colocados contiguamente uno tras otro. Los valores de columna están en el mismo orden para cada columna (la clave principal define el orden), por lo que cuando itera por muchas columnas, obtiene valores para las filas correspondientes. - -La clave principal en sí es “sparse”. No aborda cada fila, sino solo algunos rangos de datos. Separado `primary.idx` file tiene el valor de la clave principal para cada fila N-ésima, donde se llama N `index_granularity` (generalmente, N = 8192). Además, para cada columna, tenemos `column.mrk` archivos con “marks,” que son desplazamientos a cada fila N-ésima en el archivo de datos. Cada marca es un par: el desplazamiento en el archivo al comienzo del bloque comprimido y el desplazamiento en el bloque descomprimido al comienzo de los datos. Por lo general, los bloques comprimidos están alineados por marcas, y el desplazamiento en el bloque descomprimido es cero. Datos para `primary.idx` siempre reside en la memoria, y los datos para `column.mrk` archivos se almacena en caché. - -Cuando vamos a leer algo de una parte en `MergeTree` miramos `primary.idx` datos y localice rangos que podrían contener datos solicitados, luego mire `column.mrk` datos y calcular compensaciones para dónde comenzar a leer esos rangos. Debido a la escasez, el exceso de datos puede ser leído. ClickHouse no es adecuado para una gran carga de consultas de puntos simples, porque todo el rango con `index_granularity` se deben leer filas para cada clave, y todo el bloque comprimido debe descomprimirse para cada columna. Hicimos que el índice sea disperso porque debemos poder mantener billones de filas por único servidor sin un consumo de memoria notable para el índice. Además, debido a que la clave principal es escasa, no es única: no puede verificar la existencia de la clave en la tabla en el momento de INSERTAR. Podría tener muchas filas con la misma clave en una tabla. - -Cuando `INSERT` un montón de datos en `MergeTree`, ese grupo está ordenado por orden de clave primaria y forma una nueva parte. Hay subprocesos de fondo que seleccionan periódicamente algunas partes y las fusionan en una sola parte ordenada para mantener el número de partes relativamente bajo. Es por eso que se llama `MergeTree`. Por supuesto, la fusión conduce a “write amplification”. Todas las partes son inmutables: solo se crean y eliminan, pero no se modifican. Cuando se ejecuta SELECT, contiene una instantánea de la tabla (un conjunto de partes). Después de la fusión, también mantenemos las piezas viejas durante algún tiempo para facilitar la recuperación después de la falla, por lo que si vemos que alguna parte fusionada probablemente esté rota, podemos reemplazarla con sus partes de origen. - -`MergeTree` no es un árbol de LSM porque no contiene “memtable” y “log”: inserted data is written directly to the filesystem. This makes it suitable only to INSERT data in batches, not by individual row and not very frequently – about once per second is ok, but a thousand times a second is not. We did it this way for simplicity's sake, and because we are already inserting data in batches in our applications. - -> Las tablas MergeTree solo pueden tener un índice (primario): no hay índices secundarios. Sería bueno permitir múltiples representaciones físicas bajo una tabla lógica, por ejemplo, para almacenar datos en más de un orden físico o incluso para permitir representaciones con datos preagregados junto con datos originales. - -Hay motores MergeTree que están haciendo un trabajo adicional durante las fusiones en segundo plano. Los ejemplos son `CollapsingMergeTree` y `AggregatingMergeTree`. Esto podría tratarse como soporte especial para actualizaciones. Tenga en cuenta que estas no son actualizaciones reales porque los usuarios generalmente no tienen control sobre el tiempo en que se ejecutan las fusiones en segundo plano y los datos en un `MergeTree` casi siempre se almacena en más de una parte, no en forma completamente fusionada. - -## Replicación {#replication} - -La replicación en ClickHouse se puede configurar por tabla. Podría tener algunas tablas replicadas y otras no replicadas en el mismo servidor. También puede tener tablas replicadas de diferentes maneras, como una tabla con replicación de dos factores y otra con replicación de tres factores. - -La replicación se implementa en el `ReplicatedMergeTree` motor de almacenamiento. El camino en `ZooKeeper` se especifica como un parámetro para el motor de almacenamiento. Todas las tablas con la misma ruta en `ZooKeeper` se convierten en réplicas entre sí: sincronizan sus datos y mantienen la coherencia. Las réplicas se pueden agregar y eliminar dinámicamente simplemente creando o soltando una tabla. - -La replicación utiliza un esquema multi-maestro asíncrono. Puede insertar datos en cualquier réplica que tenga una sesión con `ZooKeeper`, y los datos se replican en todas las demás réplicas de forma asíncrona. Como ClickHouse no admite UPDATE, la replicación está libre de conflictos. Como no hay reconocimiento de quórum de inserciones, los datos recién insertados pueden perderse si un nodo falla. - -Los metadatos para la replicación se almacenan en ZooKeeper. Hay un registro de replicación que enumera las acciones que se deben realizar. Las acciones son: obtener parte; fusionar partes; soltar una partición, etc. Cada réplica copia el registro de replicación en su cola y, a continuación, ejecuta las acciones desde la cola. Por ejemplo, en la inserción, el “get the part” la acción se crea en el registro y cada réplica descarga esa parte. Las fusiones se coordinan entre réplicas para obtener resultados idénticos en bytes. Todas las piezas se combinan de la misma manera en todas las réplicas. Se logra eligiendo una réplica como líder, y esa réplica inicia fusiones y escrituras “merge parts” acciones al registro. - -La replicación es física: solo las partes comprimidas se transfieren entre nodos, no consultas. Las fusiones se procesan en cada réplica de forma independiente en la mayoría de los casos para reducir los costos de red al evitar la amplificación de la red. Las piezas combinadas grandes se envían a través de la red solo en casos de retraso de replicación significativo. - -Además, cada réplica almacena su estado en ZooKeeper como el conjunto de piezas y sus sumas de comprobación. Cuando el estado en el sistema de archivos local difiere del estado de referencia en ZooKeeper, la réplica restaura su coherencia descargando partes faltantes y rotas de otras réplicas. Cuando hay algunos datos inesperados o rotos en el sistema de archivos local, ClickHouse no los elimina, sino que los mueve a un directorio separado y los olvida. - -!!! note "Nota" - El clúster ClickHouse consta de fragmentos independientes y cada fragmento consta de réplicas. El clúster es **no elástico**, por lo tanto, después de agregar un nuevo fragmento, los datos no se reequilibran automáticamente entre fragmentos. En su lugar, se supone que la carga del clúster debe ajustarse para que sea desigual. Esta implementación le da más control, y está bien para clústeres relativamente pequeños, como decenas de nodos. Pero para los clústeres con cientos de nodos que estamos utilizando en producción, este enfoque se convierte en un inconveniente significativo. Debemos implementar un motor de tablas que abarque todo el clúster con regiones replicadas dinámicamente que puedan dividirse y equilibrarse entre clústeres automáticamente. - -{## [Artículo Original](https://clickhouse.tech/docs/en/development/architecture/) ##} diff --git a/docs/es/development/browse-code.md b/docs/es/development/browse-code.md deleted file mode 100644 index ca031ad03f3..00000000000 --- a/docs/es/development/browse-code.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 63 -toc_title: "Buscar c\xF3digo fuente" ---- - -# Examinar el código fuente de ClickHouse {#browse-clickhouse-source-code} - -Usted puede utilizar **Woboq** navegador de código en línea disponible [aqui](https://clickhouse.tech/codebrowser/html_report/ClickHouse/src/index.html). Proporciona navegación de código y resaltado semántico, búsqueda e indexación. La instantánea de código se actualiza diariamente. - -Además, puede navegar por las fuentes en [GitHub](https://github.com/ClickHouse/ClickHouse) como de costumbre. - -Si está interesado en qué IDE usar, recomendamos CLion, QT Creator, VS Code y KDevelop (con advertencias). Puedes usar cualquier IDE favorito. Vim y Emacs también cuentan. diff --git a/docs/es/development/build-cross-arm.md b/docs/es/development/build-cross-arm.md deleted file mode 100644 index 2758e9a0e94..00000000000 --- a/docs/es/development/build-cross-arm.md +++ /dev/null @@ -1,43 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 67 -toc_title: "C\xF3mo construir ClickHouse en Linux para AARCH64 (ARM64)" ---- - -# Cómo construir ClickHouse en Linux para la arquitectura AARCH64 (ARM64) {#how-to-build-clickhouse-on-linux-for-aarch64-arm64-architecture} - -Esto es para el caso cuando tiene una máquina Linux y desea usarla para compilar `clickhouse` binario que se ejecutará en otra máquina Linux con arquitectura de CPU AARCH64. Esto está destinado a las comprobaciones de integración continua que se ejecutan en servidores Linux. - -La compilación cruzada para AARCH64 se basa en el [Instrucciones de construcción](build.md), seguirlos primero. - -# Instalar Clang-8 {#install-clang-8} - -Siga las instrucciones de https://apt.llvm.org/ para la configuración de Ubuntu o Debian. -Por ejemplo, en Ubuntu Bionic puede usar los siguientes comandos: - -``` bash -echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" | sudo tee /etc/apt/sources.list.d/llvm.list -sudo apt-get update -sudo apt-get install clang-8 -``` - -# Instalar conjunto de herramientas de compilación cruzada {#install-cross-compilation-toolset} - -``` bash -cd ClickHouse -mkdir -p build-aarch64/cmake/toolchain/linux-aarch64 -wget 'https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en' -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C build-aarch64/cmake/toolchain/linux-aarch64 --strip-components=1 -``` - -# Construir ClickHouse {#build-clickhouse} - -``` bash -cd ClickHouse -mkdir build-arm64 -CC=clang-8 CXX=clang++-8 cmake . -Bbuild-arm64 -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake -ninja -C build-arm64 -``` - -El binario resultante se ejecutará solo en Linux con la arquitectura de CPU AARCH64. diff --git a/docs/es/development/build-cross-osx.md b/docs/es/development/build-cross-osx.md deleted file mode 100644 index d00e57c5d31..00000000000 --- a/docs/es/development/build-cross-osx.md +++ /dev/null @@ -1,64 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 66 -toc_title: "C\xF3mo construir ClickHouse en Linux para Mac OS X" ---- - -# Cómo construir ClickHouse en Linux para Mac OS X {#how-to-build-clickhouse-on-linux-for-mac-os-x} - -Esto es para el caso cuando tiene una máquina Linux y desea usarla para compilar `clickhouse` Esto está destinado a las comprobaciones de integración continuas que se ejecutan en servidores Linux. Si desea crear ClickHouse directamente en Mac OS X, continúe con [otra instrucción](build-osx.md). - -La compilación cruzada para Mac OS X se basa en el [Instrucciones de construcción](build.md), seguirlos primero. - -# Instalar Clang-8 {#install-clang-8} - -Siga las instrucciones de https://apt.llvm.org/ para la configuración de Ubuntu o Debian. -Por ejemplo, los comandos para Bionic son como: - -``` bash -sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list -sudo apt-get install clang-8 -``` - -# Instalar conjunto de herramientas de compilación cruzada {#install-cross-compilation-toolset} - -Recordemos la ruta donde instalamos `cctools` como ${CCTOOLS} - -``` bash -mkdir ${CCTOOLS} - -git clone https://github.com/tpoechtrager/apple-libtapi.git -cd apple-libtapi -INSTALLPREFIX=${CCTOOLS} ./build.sh -./install.sh -cd .. - -git clone https://github.com/tpoechtrager/cctools-port.git -cd cctools-port/cctools -./configure --prefix=${CCTOOLS} --with-libtapi=${CCTOOLS} --target=x86_64-apple-darwin -make install -``` - -Además, necesitamos descargar macOS X SDK en el árbol de trabajo. - -``` bash -cd ClickHouse -wget 'https://github.com/phracker/MacOSX-SDKs/releases/download/10.15/MacOSX10.15.sdk.tar.xz' -mkdir -p build-darwin/cmake/toolchain/darwin-x86_64 -tar xJf MacOSX10.15.sdk.tar.xz -C build-darwin/cmake/toolchain/darwin-x86_64 --strip-components=1 -``` - -# Construir ClickHouse {#build-clickhouse} - -``` bash -cd ClickHouse -mkdir build-osx -CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake \ - -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \ - -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \ - -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -ninja -C build-osx -``` - -El binario resultante tendrá un formato ejecutable Mach-O y no se puede ejecutar en Linux. diff --git a/docs/es/development/build-osx.md b/docs/es/development/build-osx.md deleted file mode 100644 index 39eba389798..00000000000 --- a/docs/es/development/build-osx.md +++ /dev/null @@ -1,93 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 65 -toc_title: "C\xF3mo crear ClickHouse en Mac OS X" ---- - -# Cómo crear ClickHouse en Mac OS X {#how-to-build-clickhouse-on-mac-os-x} - -Build debería funcionar en Mac OS X 10.15 (Catalina) - -## Instalar Homebrew {#install-homebrew} - -``` bash -$ /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" -``` - -## Instalar compiladores, herramientas y bibliotecas necesarios {#install-required-compilers-tools-and-libraries} - -``` bash -$ brew install cmake ninja libtool gettext -``` - -## Fuentes de ClickHouse de pago {#checkout-clickhouse-sources} - -``` bash -$ git clone --recursive git@github.com:ClickHouse/ClickHouse.git -``` - -o - -``` bash -$ git clone --recursive https://github.com/ClickHouse/ClickHouse.git - -$ cd ClickHouse -``` - -## Construir ClickHouse {#build-clickhouse} - -``` bash -$ mkdir build -$ cd build -$ cmake .. -DCMAKE_CXX_COMPILER=`which clang++` -DCMAKE_C_COMPILER=`which clang` -$ ninja -$ cd .. -``` - -## Advertencia {#caveats} - -Si tiene la intención de ejecutar clickhouse-server, asegúrese de aumentar la variable maxfiles del sistema. - -!!! info "Nota" - Tendrás que usar sudo. - -Para ello, cree el siguiente archivo: - -/Library/LaunchDaemons/limit.maxfiles.lista: - -``` xml - - - - - Label - limit.maxfiles - ProgramArguments - - launchctl - limit - maxfiles - 524288 - 524288 - - RunAtLoad - - ServiceIPC - - - -``` - -Ejecute el siguiente comando: - -``` bash -$ sudo chown root:wheel /Library/LaunchDaemons/limit.maxfiles.plist -``` - -Reiniciar. - -Para verificar si está funcionando, puede usar `ulimit -n` comando. - -[Artículo Original](https://clickhouse.tech/docs/en/development/build_osx/) diff --git a/docs/es/development/build.md b/docs/es/development/build.md deleted file mode 100644 index 42cd9b5433f..00000000000 --- a/docs/es/development/build.md +++ /dev/null @@ -1,141 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 64 -toc_title: "C\xF3mo crear ClickHouse en Linux" ---- - -# Cómo construir ClickHouse para el desarrollo {#how-to-build-clickhouse-for-development} - -El siguiente tutorial se basa en el sistema Ubuntu Linux. -Con los cambios apropiados, también debería funcionar en cualquier otra distribución de Linux. -Plataformas compatibles: x86_64 y AArch64. El soporte para Power9 es experimental. - -## Instalar Git, CMake, Python y Ninja {#install-git-cmake-python-and-ninja} - -``` bash -$ sudo apt-get install git cmake python ninja-build -``` - -O cmake3 en lugar de cmake en sistemas más antiguos. - -## Instalar GCC 10 {#install-gcc-10} - -Hay varias formas de hacer esto. - -### Instalar desde un paquete PPA {#install-from-a-ppa-package} - -``` bash -$ sudo apt-get install software-properties-common -$ sudo apt-add-repository ppa:ubuntu-toolchain-r/test -$ sudo apt-get update -$ sudo apt-get install gcc-10 g++-10 -``` - -### Instalar desde fuentes {#install-from-sources} - -Mira [Sistema abierto.](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) - -## Usar GCC 10 para compilaciones {#use-gcc-10-for-builds} - -``` bash -$ export CC=gcc-10 -$ export CXX=g++-10 -``` - -## Fuentes de ClickHouse de pago {#checkout-clickhouse-sources} - -``` bash -$ git clone --recursive git@github.com:ClickHouse/ClickHouse.git -``` - -o - -``` bash -$ git clone --recursive https://github.com/ClickHouse/ClickHouse.git -``` - -## Construir ClickHouse {#build-clickhouse} - -``` bash -$ cd ClickHouse -$ mkdir build -$ cd build -$ cmake .. -$ ninja -$ cd .. -``` - -Para crear un ejecutable, ejecute `ninja clickhouse`. -Esto creará el `programs/clickhouse` ejecutable, que se puede usar con `client` o `server` argumento. - -# Cómo construir ClickHouse en cualquier Linux {#how-to-build-clickhouse-on-any-linux} - -La compilación requiere los siguientes componentes: - -- Git (se usa solo para verificar las fuentes, no es necesario para la compilación) -- CMake 3.10 o más reciente -- Ninja (recomendado) o Hacer -- Compilador de C ++: gcc 10 o clang 8 o más reciente -- Enlazador: lld u oro (el clásico GNU ld no funcionará) -- Python (solo se usa dentro de la compilación LLVM y es opcional) - -Si todos los componentes están instalados, puede compilar de la misma manera que los pasos anteriores. - -Ejemplo para Ubuntu Eoan: - - sudo apt update - sudo apt install git cmake ninja-build g++ python - git clone --recursive https://github.com/ClickHouse/ClickHouse.git - mkdir build && cd build - cmake ../ClickHouse - ninja - -Ejemplo de OpenSUSE Tumbleweed: - - sudo zypper install git cmake ninja gcc-c++ python lld - git clone --recursive https://github.com/ClickHouse/ClickHouse.git - mkdir build && cd build - cmake ../ClickHouse - ninja - -Ejemplo de Fedora Rawhide: - - sudo yum update - yum --nogpg install git cmake make gcc-c++ python3 - git clone --recursive https://github.com/ClickHouse/ClickHouse.git - mkdir build && cd build - cmake ../ClickHouse - make -j $(nproc) - -# No tienes que construir ClickHouse {#you-dont-have-to-build-clickhouse} - -ClickHouse está disponible en binarios y paquetes preconstruidos. Los binarios son portátiles y se pueden ejecutar en cualquier tipo de Linux. - -Están diseñados para lanzamientos estables, preestablecidos y de prueba, siempre que para cada compromiso con el maestro y para cada solicitud de extracción. - -Para encontrar la construcción más fresca de `master`, ir a [se compromete página](https://github.com/ClickHouse/ClickHouse/commits/master), haga clic en la primera marca de verificación verde o cruz roja cerca de confirmar, y haga clic en “Details” enlace justo después “ClickHouse Build Check”. - -# Cómo construir el paquete Debian ClickHouse {#how-to-build-clickhouse-debian-package} - -## Instalar Git y Pbuilder {#install-git-and-pbuilder} - -``` bash -$ sudo apt-get update -$ sudo apt-get install git python pbuilder debhelper lsb-release fakeroot sudo debian-archive-keyring debian-keyring -``` - -## Fuentes de ClickHouse de pago {#checkout-clickhouse-sources-1} - -``` bash -$ git clone --recursive --branch master https://github.com/ClickHouse/ClickHouse.git -$ cd ClickHouse -``` - -## Ejecutar secuencia de comandos de lanzamiento {#run-release-script} - -``` bash -$ ./release -``` - -[Artículo Original](https://clickhouse.tech/docs/en/development/build/) diff --git a/docs/es/development/contrib.md b/docs/es/development/contrib.md deleted file mode 100644 index 3f3013570e5..00000000000 --- a/docs/es/development/contrib.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 70 -toc_title: Bibliotecas de terceros utilizadas ---- - -# Bibliotecas de terceros utilizadas {#third-party-libraries-used} - -| Biblioteca | Licencia | -|--------------------|--------------------------------------------------------------------------------------------------------------------------------------------------| -| base64 | [Licencia BSD de 2 cláusulas](https://github.com/aklomp/base64/blob/a27c565d1b6c676beaf297fe503c4518185666f7/LICENSE) | -| impulsar | [Licencia de software Boost 1.0](https://github.com/ClickHouse-Extras/boost-extra/blob/6883b40449f378019aec792f9983ce3afc7ff16e/LICENSE_1_0.txt) | -| Bienvenido | [MIT](https://github.com/google/brotli/blob/master/LICENSE) | -| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/master/LICENSE) | -| Cctz | [Licencia Apache 2.0](https://github.com/google/cctz/blob/4f9776a310f4952454636363def82c2bf6641d5f/LICENSE.txt) | -| doble conversión | [Licencia de 3 cláusulas BSD](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) | -| FastMemcpy | [MIT](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libmemcpy/impl/LICENSE) | -| Más información | [Licencia de 3 cláusulas BSD](https://github.com/google/googletest/blob/master/LICENSE) | -| H3 | [Licencia Apache 2.0](https://github.com/uber/h3/blob/master/LICENSE) | -| hyperscan | [Licencia de 3 cláusulas BSD](https://github.com/intel/hyperscan/blob/master/LICENSE) | -| libcxxabi | [BSD + MIT](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT) | -| libdivide | [Licencia Zlib](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) | -| libgsasl | [Información adicional](https://github.com/ClickHouse-Extras/libgsasl/blob/3b8948a4042e34fb00b4fb987535dc9e02e39040/LICENSE) | -| libhdfs3 | [Licencia Apache 2.0](https://github.com/ClickHouse-Extras/libhdfs3/blob/bd6505cbb0c130b0db695305b9a38546fa880e5a/LICENSE.txt) | -| libmetrohash | [Licencia Apache 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) | -| libpcg-al azar | [Licencia Apache 2.0](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | -| Libressl | [Licencia OpenSSL](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | -| Librdkafka | [Licencia BSD de 2 cláusulas](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | -| libwidechar_width | [CC0 1.0 Universal](https://github.com/ClickHouse/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | -| llvm | [Licencia de 3 cláusulas BSD](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | -| lz4 | [Licencia BSD de 2 cláusulas](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | -| mariadb-conector-c | [Información adicional](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | -| murmurhash | [Dominio público](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/murmurhash/LICENSE) | -| pdqsort | [Licencia Zlib](https://github.com/ClickHouse/ClickHouse/blob/master/contrib/pdqsort/license.txt) | -| Poco | [Boost Software License - Versión 1.0](https://github.com/ClickHouse-Extras/poco/blob/fe5505e56c27b6ecb0dcbc40c49dc2caf4e9637f/LICENSE) | -| protobuf | [Licencia de 3 cláusulas BSD](https://github.com/ClickHouse-Extras/protobuf/blob/12735370922a35f03999afff478e1c6d7aa917a4/LICENSE) | -| Re2 | [Licencia de 3 cláusulas BSD](https://github.com/google/re2/blob/7cf8b88e8f70f97fd4926b56aa87e7f53b2717e0/LICENSE) | -| UnixODBC | [Información adicional](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) | -| Sistema abierto. | [Licencia Zlib](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) | -| zstd | [Licencia de 3 cláusulas BSD](https://github.com/facebook/zstd/blob/dev/LICENSE) | diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md deleted file mode 100644 index 0ce5d0b457a..00000000000 --- a/docs/es/development/developer-instruction.md +++ /dev/null @@ -1,287 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 61 -toc_title: "La instrucci\xF3n para desarrolladores de ClickHouse para principiantes" ---- - -La construcción de ClickHouse es compatible con Linux, FreeBSD y Mac OS X. - -# Si utiliza Windows {#if-you-use-windows} - -Si usa Windows, necesita crear una máquina virtual con Ubuntu. Para comenzar a trabajar con una máquina virtual, instale VirtualBox. Puede descargar Ubuntu desde el sitio web: https://www.ubuntu.com/#download. Por favor, cree una máquina virtual a partir de la imagen descargada (debe reservar al menos 4 GB de RAM para ello). Para ejecutar un terminal de línea de comandos en Ubuntu, busque un programa que contenga la palabra “terminal” en su nombre (gnome-terminal, konsole etc.) o simplemente presione Ctrl + Alt + T. - -# Si utiliza un sistema de 32 bits {#if-you-use-a-32-bit-system} - -ClickHouse no puede funcionar ni construir en un sistema de 32 bits. Debe adquirir acceso a un sistema de 64 bits y puede continuar leyendo. - -# Creación de un repositorio en GitHub {#creating-a-repository-on-github} - -Para comenzar a trabajar con el repositorio de ClickHouse, necesitará una cuenta de GitHub. - -Probablemente ya tenga uno, pero si no lo hace, regístrese en https://github.com . En caso de que no tenga claves SSH, debe generarlas y luego cargarlas en GitHub. Es necesario para enviar a través de sus parches. También es posible usar las mismas claves SSH que usa con cualquier otro servidor SSH, probablemente ya las tenga. - -Cree una bifurcación del repositorio ClickHouse. Para hacerlo por favor haga clic en el “fork” botón en la esquina superior derecha en https://github.com/ClickHouse/ClickHouse . Se bifurcará su propia copia de ClickHouse/ClickHouse a su cuenta. - -El proceso de desarrollo consiste en comprometer primero los cambios previstos en su bifurcación de ClickHouse y luego crear un “pull request” para que estos cambios sean aceptados en el repositorio principal (ClickHouse / ClickHouse). - -Para trabajar con repositorios git, instale `git`. - -Para hacer eso en Ubuntu, ejecutaría en la terminal de línea de comandos: - - sudo apt update - sudo apt install git - -Puede encontrar un breve manual sobre el uso de Git aquí: https://education.github.com/git-cheat-sheet-education.pdf . -Para obtener un manual detallado sobre Git, consulte https://git-scm.com/book/en/v2 . - -# Clonación de un repositorio en su máquina de desarrollo {#cloning-a-repository-to-your-development-machine} - -A continuación, debe descargar los archivos fuente en su máquina de trabajo. Esto se llama “to clone a repository” porque crea una copia local del repositorio en su máquina de trabajo. - -En el terminal de línea de comandos, ejecute: - - git clone --recursive git@github.com:your_github_username/ClickHouse.git - cd ClickHouse - -Nota: por favor, sustituye *your_github_username* con lo que es apropiado! - -Este comando creará un directorio `ClickHouse` que contiene la copia de trabajo del proyecto. - -Es importante que la ruta al directorio de trabajo no contenga espacios en blanco, ya que puede ocasionar problemas con la ejecución del sistema de compilación. - -Tenga en cuenta que el repositorio ClickHouse utiliza `submodules`. That is what the references to additional repositories are called (i.e. external libraries on which the project depends). It means that when cloning the repository you need to specify the `--recursive` como en el ejemplo anterior. Si el repositorio se ha clonado sin submódulos, para descargarlos debe ejecutar lo siguiente: - - git submodule init - git submodule update - -Puede verificar el estado con el comando: `git submodule status`. - -Si recibe el siguiente mensaje de error: - - Permission denied (publickey). - fatal: Could not read from remote repository. - - Please make sure you have the correct access rights - and the repository exists. - -Por lo general, significa que faltan las claves SSH para conectarse a GitHub. Estas teclas se encuentran normalmente en `~/.ssh`. Para que las claves SSH sean aceptadas, debe cargarlas en la sección de configuración de la interfaz de usuario de GitHub. - -También puede clonar el repositorio a través del protocolo https: - - git clone https://github.com/ClickHouse/ClickHouse.git - -Sin embargo, esto no le permitirá enviar los cambios al servidor. Aún puede usarlo temporalmente y agregar las claves SSH más tarde reemplazando la dirección remota del repositorio con `git remote` comando. - -También puede agregar la dirección original del repositorio de ClickHouse a su repositorio local para extraer actualizaciones desde allí: - - git remote add upstream git@github.com:ClickHouse/ClickHouse.git - -Después de ejecutar con éxito este comando, podrá extraer actualizaciones del repositorio principal de ClickHouse ejecutando `git pull upstream master`. - -## Trabajar con submódulos {#working-with-submodules} - -Trabajar con submódulos en git podría ser doloroso. Los siguientes comandos ayudarán a administrarlo: - - # ! each command accepts --recursive - # Update remote URLs for submodules. Barely rare case - git submodule sync - # Add new submodules - git submodule init - # Update existing submodules to the current state - git submodule update - # Two last commands could be merged together - git submodule update --init - -Los siguientes comandos le ayudarían a restablecer todos los submódulos al estado inicial (!¡ADVERTENCIA! - cualquier cambio en el interior será eliminado): - - # Synchronizes submodules' remote URL with .gitmodules - git submodule sync --recursive - # Update the registered submodules with initialize not yet initialized - git submodule update --init --recursive - # Reset all changes done after HEAD - git submodule foreach git reset --hard - # Clean files from .gitignore - git submodule foreach git clean -xfd - # Repeat last 4 commands for all submodule - git submodule foreach git submodule sync --recursive - git submodule foreach git submodule update --init --recursive - git submodule foreach git submodule foreach git reset --hard - git submodule foreach git submodule foreach git clean -xfd - -# Sistema de construcción {#build-system} - -ClickHouse utiliza CMake y Ninja para la construcción. - -CMake - un sistema de meta-construcción que puede generar archivos Ninja (tareas de construcción). -Ninja: un sistema de compilación más pequeño con un enfoque en la velocidad utilizada para ejecutar esas tareas generadas por cmake. - -Para instalar en Ubuntu, Debian o Mint run `sudo apt install cmake ninja-build`. - -En CentOS, RedHat se ejecuta `sudo yum install cmake ninja-build`. - -Si usa Arch o Gentoo, probablemente lo sepa usted mismo cómo instalar CMake. - -Para instalar CMake y Ninja en Mac OS X, primero instale Homebrew y luego instale todo lo demás a través de brew: - - /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" - brew install cmake ninja - -A continuación, verifique la versión de CMake: `cmake --version`. Si está por debajo de 3.3, debe instalar una versión más reciente desde el sitio web: https://cmake.org/download/. - -# Bibliotecas externas opcionales {#optional-external-libraries} - -ClickHouse utiliza varias bibliotecas externas para la construcción. Todos ellos no necesitan ser instalados por separado, ya que se construyen junto con ClickHouse a partir de las fuentes ubicadas en los submódulos. Puede consultar la lista en `contrib`. - -# Compilador de C ++ {#c-compiler} - -Los compiladores GCC a partir de la versión 10 y Clang versión 8 o superior son compatibles para construir ClickHouse. - -Las compilaciones oficiales de Yandex actualmente usan GCC porque genera código de máquina de un rendimiento ligeramente mejor (con una diferencia de hasta varios por ciento según nuestros puntos de referencia). Y Clang es más conveniente para el desarrollo generalmente. Sin embargo, nuestra plataforma de integración continua (CI) ejecuta verificaciones de aproximadamente una docena de combinaciones de compilación. - -Para instalar GCC en Ubuntu, ejecute: `sudo apt install gcc g++` - -Compruebe la versión de gcc: `gcc --version`. Si está por debajo de 9, siga las instrucciones aquí: https://clickhouse.tech/docs/es/development/build/#install-gcc-10. - -La compilación de Mac OS X solo es compatible con Clang. Sólo tiene que ejecutar `brew install llvm` - -Si decide utilizar Clang, también puede instalar `libc++` y `lld` si usted sabe lo que es. Utilizar `ccache` también se recomienda. - -# El proceso de construcción {#the-building-process} - -Ahora que está listo para construir ClickHouse, le recomendamos que cree un directorio separado `build` dentro `ClickHouse` que contendrá todos los de la generación de artefactos: - - mkdir build - cd build - -Puede tener varios directorios diferentes (build_release, build_debug, etc.) para diferentes tipos de construcción. - -Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 10 en este ejemplo). - -Linux: - - export CC=gcc-10 CXX=g++-10 - cmake .. - -Mac OS X: - - export CC=clang CXX=clang++ - cmake .. - -El `CC` variable especifica el compilador para C (abreviatura de C Compiler), y `CXX` variable indica qué compilador de C ++ se usará para compilar. - -Para una construcción más rápida, puede recurrir al `debug` tipo de compilación: una compilación sin optimizaciones. Para ese suministro el siguiente parámetro `-D CMAKE_BUILD_TYPE=Debug`: - - cmake -D CMAKE_BUILD_TYPE=Debug .. - -Puede cambiar el tipo de compilación ejecutando este comando en el `build` directorio. - -Ejecutar ninja para construir: - - ninja clickhouse-server clickhouse-client - -Solo los binarios requeridos se van a construir en este ejemplo. - -Si necesita construir todos los binarios (utilidades y pruebas), debe ejecutar ninja sin parámetros: - - ninja - -La compilación completa requiere aproximadamente 30 GB de espacio libre en disco o 15 GB para construir los binarios principales. - -Cuando hay una gran cantidad de RAM disponible en la máquina de compilación, debe limitar el número de tareas de compilación que se ejecutan en paralelo con `-j` parámetro: - - ninja -j 1 clickhouse-server clickhouse-client - -En máquinas con 4GB de RAM, se recomienda especificar 1, para 8GB de RAM `-j 2` se recomienda. - -Si recibe el mensaje: `ninja: error: loading 'build.ninja': No such file or directory`, significa que la generación de una configuración de compilación ha fallado y necesita inspeccionar el mensaje anterior. - -Cuando se inicie correctamente el proceso de construcción, verá el progreso de la compilación: el número de tareas procesadas y el número total de tareas. - -Al crear mensajes sobre archivos protobuf en la biblioteca libhdfs2, como `libprotobuf WARNING` puede aparecer. Afectan a nada y son seguros para ser ignorado. - -Tras la compilación exitosa, obtienes un archivo ejecutable `ClickHouse//programs/clickhouse`: - - ls -l programs/clickhouse - -# Ejecución del ejecutable construido de ClickHouse {#running-the-built-executable-of-clickhouse} - -Para ejecutar el servidor bajo el usuario actual, debe navegar hasta `ClickHouse/programs/server/` (situado fuera de `build`) y ejecutar: - - ../../build/programs/clickhouse server - -En este caso, ClickHouse usará archivos de configuración ubicados en el directorio actual. Puede ejecutar `clickhouse server` desde cualquier directorio que especifique la ruta a un archivo de configuración como un parámetro de línea de comandos `--config-file`. - -Para conectarse a ClickHouse con clickhouse-client en otro terminal, vaya a `ClickHouse/build/programs/` y ejecutar `./clickhouse client`. - -Si usted consigue `Connection refused` mensaje en Mac OS X o FreeBSD, intente especificar la dirección de host 127.0.0.1: - - clickhouse client --host 127.0.0.1 - -Puede reemplazar la versión de producción del binario ClickHouse instalado en su sistema con su binario ClickHouse personalizado. Para ello, instale ClickHouse en su máquina siguiendo las instrucciones del sitio web oficial. A continuación, ejecute lo siguiente: - - sudo service clickhouse-server stop - sudo cp ClickHouse/build/programs/clickhouse /usr/bin/ - sudo service clickhouse-server start - -Tenga en cuenta que `clickhouse-client`, `clickhouse-server` y otros son enlaces simbólicos a los comúnmente compartidos `clickhouse` binario. - -También puede ejecutar su binario ClickHouse personalizado con el archivo de configuración del paquete ClickHouse instalado en su sistema: - - sudo service clickhouse-server stop - sudo -u clickhouse ClickHouse/build/programs/clickhouse server --config-file /etc/clickhouse-server/config.xml - -# IDE (entorno de desarrollo integrado) {#ide-integrated-development-environment} - -Si no sabe qué IDE usar, le recomendamos que use CLion. CLion es un software comercial, pero ofrece un período de prueba gratuito de 30 días. También es gratuito para los estudiantes. CLion se puede usar tanto en Linux como en Mac OS X. - -KDevelop y QTCreator son otras excelentes alternativas de un IDE para desarrollar ClickHouse. KDevelop viene como un IDE muy útil aunque inestable. Si KDevelop se bloquea después de un tiempo al abrir el proyecto, debe hacer clic “Stop All” botón tan pronto como se ha abierto la lista de archivos del proyecto. Después de hacerlo, KDevelop debería estar bien para trabajar. - -Como editores de código simples, puede usar Sublime Text o Visual Studio Code, o Kate (todos los cuales están disponibles en Linux). - -Por si acaso, vale la pena mencionar que CLion crea `build` por sí mismo, también por sí mismo selecciona `debug` para el tipo de compilación, para la configuración usa una versión de CMake que está definida en CLion y no la instalada por usted, y finalmente, CLion usará `make` para ejecutar tareas de compilación en lugar de `ninja`. Este es un comportamiento normal, solo tenlo en cuenta para evitar confusiones. - -# Código de escritura {#writing-code} - -La descripción de la arquitectura ClickHouse se puede encontrar aquí: https://clickhouse.tech/docs/es/desarrollo/arquitectura/ - -La Guía de estilo de código: https://clickhouse.tech/docs/en/development/style/ - -Pruebas de escritura: https://clickhouse.tech/docs/en/development/tests/ - -Lista de tareas: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 - -# Datos de prueba {#test-data} - -El desarrollo de ClickHouse a menudo requiere cargar conjuntos de datos realistas. Es particularmente importante para las pruebas de rendimiento. Tenemos un conjunto especialmente preparado de datos anónimos de Yandex.Métrica. Se requiere, además, unos 3 GB de espacio libre en disco. Tenga en cuenta que estos datos no son necesarios para realizar la mayoría de las tareas de desarrollo. - - sudo apt install wget xz-utils - - wget https://datasets.clickhouse.tech/hits/tsv/hits_v1.tsv.xz - wget https://datasets.clickhouse.tech/visits/tsv/visits_v1.tsv.xz - - xz -v -d hits_v1.tsv.xz - xz -v -d visits_v1.tsv.xz - - clickhouse-client - - CREATE DATABASE IF NOT EXISTS test - - CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime); - - CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID); - - clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv - clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv - -# Creación de solicitud de extracción {#creating-pull-request} - -Navega a tu repositorio de fork en la interfaz de usuario de GitHub. Si ha estado desarrollando en una sucursal, debe seleccionar esa sucursal. Habrá un “Pull request” botón situado en la pantalla. En esencia, esto significa “create a request for accepting my changes into the main repository”. - -Se puede crear una solicitud de extracción incluso si el trabajo aún no se ha completado. En este caso, por favor ponga la palabra “WIP” (trabajo en curso) al comienzo del título, se puede cambiar más tarde. Esto es útil para la revisión cooperativa y la discusión de los cambios, así como para ejecutar todas las pruebas disponibles. Es importante que proporcione una breve descripción de sus cambios, que más tarde se utilizará para generar registros de cambios de lanzamiento. - -Las pruebas comenzarán tan pronto como los empleados de Yandex etiqueten su PR con una etiqueta “can be tested”. The results of some first checks (e.g. code style) will come in within several minutes. Build check results will arrive within half an hour. And the main set of tests will report itself within an hour. - -El sistema preparará compilaciones binarias ClickHouse para su solicitud de extracción individualmente. Para recuperar estas compilaciones, haga clic en “Details” junto al link “ClickHouse build check” en la lista de cheques. Allí encontrará enlaces directos a la construcción.deb paquetes de ClickHouse que puede implementar incluso en sus servidores de producción (si no tiene miedo). - -Lo más probable es que algunas de las compilaciones fallen las primeras veces. Esto se debe al hecho de que verificamos las compilaciones tanto con gcc como con clang, con casi todas las advertencias existentes (siempre con el `-Werror` bandera) habilitado para sonido. En esa misma página, puede encontrar todos los registros de compilación para que no tenga que compilar ClickHouse de todas las formas posibles. diff --git a/docs/es/development/index.md b/docs/es/development/index.md deleted file mode 100644 index 6f96f9b3f02..00000000000 --- a/docs/es/development/index.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Desarrollo -toc_hidden: true -toc_priority: 58 -toc_title: oculto ---- - -# Desarrollo de ClickHouse {#clickhouse-development} - -[Artículo Original](https://clickhouse.tech/docs/en/development/) diff --git a/docs/es/development/style.md b/docs/es/development/style.md deleted file mode 100644 index ec55516fe2c..00000000000 --- a/docs/es/development/style.md +++ /dev/null @@ -1,841 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 68 -toc_title: "C\xF3mo escribir c\xF3digo C ++" ---- - -# Cómo escribir código C ++ {#how-to-write-c-code} - -## Recomendaciones generales {#general-recommendations} - -**1.** Las siguientes son recomendaciones, no requisitos. - -**2.** Si está editando código, tiene sentido seguir el formato del código existente. - -**3.** El estilo de código es necesario para la coherencia. La consistencia facilita la lectura del código y también facilita la búsqueda del código. - -**4.** Muchas de las reglas no tienen razones lógicas; están dictadas por prácticas establecidas. - -## Formatear {#formatting} - -**1.** La mayor parte del formato se realizará automáticamente por `clang-format`. - -**2.** Las sangrías son 4 espacios. Configure el entorno de desarrollo para que una pestaña agregue cuatro espacios. - -**3.** Abrir y cerrar llaves deben estar en una línea separada. - -``` cpp -inline void readBoolText(bool & x, ReadBuffer & buf) -{ - char tmp = '0'; - readChar(tmp, buf); - x = tmp != '0'; -} -``` - -**4.** Si todo el cuerpo de la función es `statement`, se puede colocar en una sola línea. Coloque espacios alrededor de llaves (además del espacio al final de la línea). - -``` cpp -inline size_t mask() const { return buf_size() - 1; } -inline size_t place(HashValue x) const { return x & mask(); } -``` - -**5.** Para funciones. No coloque espacios alrededor de los corchetes. - -``` cpp -void reinsert(const Value & x) -``` - -``` cpp -memcpy(&buf[place_value], &x, sizeof(x)); -``` - -**6.** En `if`, `for`, `while` y otras expresiones, se inserta un espacio delante del corchete de apertura (a diferencia de las llamadas a funciones). - -``` cpp -for (size_t i = 0; i < rows; i += storage.index_granularity) -``` - -**7.** Agregar espacios alrededor de los operadores binarios (`+`, `-`, `*`, `/`, `%`, …) and the ternary operator `?:`. - -``` cpp -UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); -UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); -UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); -``` - -**8.** Si se introduce un avance de línea, coloque al operador en una nueva línea y aumente la sangría antes de ella. - -``` cpp -if (elapsed_ns) - message << " (" - << rows_read_on_server * 1000000000 / elapsed_ns << " rows/s., " - << bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) "; -``` - -**9.** Puede utilizar espacios para la alineación dentro de una línea, si lo desea. - -``` cpp -dst.ClickLogID = click.LogID; -dst.ClickEventID = click.EventID; -dst.ClickGoodEvent = click.GoodEvent; -``` - -**10.** No use espacios alrededor de los operadores `.`, `->`. - -Si es necesario, el operador se puede envolver a la siguiente línea. En este caso, el desplazamiento frente a él aumenta. - -**11.** No utilice un espacio para separar los operadores unarios (`--`, `++`, `*`, `&`, …) from the argument. - -**12.** Pon un espacio después de una coma, pero no antes. La misma regla se aplica a un punto y coma dentro de un `for` expresion. - -**13.** No utilice espacios para separar el `[]` operador. - -**14.** En un `template <...>` expresión, use un espacio entre `template` y `<`; sin espacios después de `<` o antes `>`. - -``` cpp -template -struct AggregatedStatElement -{} -``` - -**15.** En clases y estructuras, escribe `public`, `private`, y `protected` en el mismo nivel que `class/struct`, y sangrar el resto del código. - -``` cpp -template -class MultiVersion -{ -public: - /// Version of object for usage. shared_ptr manage lifetime of version. - using Version = std::shared_ptr; - ... -} -``` - -**16.** Si el mismo `namespace` se usa para todo el archivo, y no hay nada más significativo, no es necesario un desplazamiento dentro `namespace`. - -**17.** Si el bloque para un `if`, `for`, `while`, u otra expresión consiste en una sola `statement`, las llaves son opcionales. Coloque el `statement` en una línea separada, en su lugar. Esta regla también es válida para `if`, `for`, `while`, … - -Pero si el interior `statement` contiene llaves o `else`, el bloque externo debe escribirse entre llaves. - -``` cpp -/// Finish write. -for (auto & stream : streams) - stream.second->finalize(); -``` - -**18.** No debería haber espacios al final de las líneas. - -**19.** Los archivos de origen están codificados en UTF-8. - -**20.** Los caracteres no ASCII se pueden usar en literales de cadena. - -``` cpp -<< ", " << (timer.elapsed() / chunks_stats.hits) << " μsec/hit."; -``` - -**21.** No escriba varias expresiones en una sola línea. - -**22.** Agrupe secciones de código dentro de las funciones y sepárelas con no más de una línea vacía. - -**23.** Separe funciones, clases, etc. con una o dos líneas vacías. - -**24.** `A const` (relacionado con un valor) debe escribirse antes del nombre del tipo. - -``` cpp -//correct -const char * pos -const std::string & s -//incorrect -char const * pos -``` - -**25.** Al declarar un puntero o referencia, el `*` y `&` Los símbolos deben estar separados por espacios en ambos lados. - -``` cpp -//correct -const char * pos -//incorrect -const char* pos -const char *pos -``` - -**26.** Cuando utilice tipos de plantilla, alias con el `using` palabra clave (excepto en los casos más simples). - -En otras palabras, los parámetros de la plantilla se especifican solo en `using` y no se repiten en el código. - -`using` se puede declarar localmente, como dentro de una función. - -``` cpp -//correct -using FileStreams = std::map>; -FileStreams streams; -//incorrect -std::map> streams; -``` - -**27.** No declare varias variables de diferentes tipos en una instrucción. - -``` cpp -//incorrect -int x, *y; -``` - -**28.** No utilice moldes de estilo C. - -``` cpp -//incorrect -std::cerr << (int)c <<; std::endl; -//correct -std::cerr << static_cast(c) << std::endl; -``` - -**29.** En clases y estructuras, los miembros del grupo y las funciones por separado dentro de cada ámbito de visibilidad. - -**30.** Para clases y estructuras pequeñas, no es necesario separar la declaración del método de la implementación. - -Lo mismo es cierto para los métodos pequeños en cualquier clase o estructura. - -Para clases y estructuras con plantillas, no separe las declaraciones de métodos de la implementación (porque de lo contrario deben definirse en la misma unidad de traducción). - -**31.** Puede ajustar líneas en 140 caracteres, en lugar de 80. - -**32.** Utilice siempre los operadores de incremento / decremento de prefijo si no se requiere postfix. - -``` cpp -for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it) -``` - -## Comentario {#comments} - -**1.** Asegúrese de agregar comentarios para todas las partes no triviales del código. - -Esto es muy importante. Escribir el comentario puede ayudarte a darte cuenta de que el código no es necesario o que está diseñado incorrectamente. - -``` cpp -/** Part of piece of memory, that can be used. - * For example, if internal_buffer is 1MB, and there was only 10 bytes loaded to buffer from file for reading, - * then working_buffer will have size of only 10 bytes - * (working_buffer.end() will point to position right after those 10 bytes available for read). - */ -``` - -**2.** Los comentarios pueden ser tan detallados como sea necesario. - -**3.** Coloque comentarios antes del código que describen. En casos raros, los comentarios pueden aparecer después del código, en la misma línea. - -``` cpp -/** Parses and executes the query. -*/ -void executeQuery( - ReadBuffer & istr, /// Where to read the query from (and data for INSERT, if applicable) - WriteBuffer & ostr, /// Where to write the result - Context & context, /// DB, tables, data types, engines, functions, aggregate functions... - BlockInputStreamPtr & query_plan, /// Here could be written the description on how query was executed - QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// Up to which stage process the SELECT query - ) -``` - -**4.** Los comentarios deben escribirse en inglés solamente. - -**5.** Si está escribiendo una biblioteca, incluya comentarios detallados que la expliquen en el archivo de encabezado principal. - -**6.** No agregue comentarios que no proporcionen información adicional. En particular, no deje comentarios vacíos como este: - -``` cpp -/* -* Procedure Name: -* Original procedure name: -* Author: -* Date of creation: -* Dates of modification: -* Modification authors: -* Original file name: -* Purpose: -* Intent: -* Designation: -* Classes used: -* Constants: -* Local variables: -* Parameters: -* Date of creation: -* Purpose: -*/ -``` - -El ejemplo se toma prestado del recurso http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/. - -**7.** No escriba comentarios de basura (autor, fecha de creación ..) al principio de cada archivo. - -**8.** Los comentarios de una sola línea comienzan con tres barras: `///` y los comentarios de varias líneas comienzan con `/**`. Estos comentarios son considerados “documentation”. - -Nota: Puede usar Doxygen para generar documentación a partir de estos comentarios. Pero Doxygen no se usa generalmente porque es más conveniente navegar por el código en el IDE. - -**9.** Los comentarios de varias líneas no deben tener líneas vacías al principio y al final (excepto la línea que cierra un comentario de varias líneas). - -**10.** Para comentar el código, use comentarios básicos, no “documenting” comentario. - -**11.** Elimine las partes comentadas del código antes de confirmar. - -**12.** No use blasfemias en comentarios o código. - -**13.** No use letras mayúsculas. No use puntuación excesiva. - -``` cpp -/// WHAT THE FAIL??? -``` - -**14.** No use comentarios para hacer delímetros. - -``` cpp -///****************************************************** -``` - -**15.** No comiencen las discusiones en los comentarios. - -``` cpp -/// Why did you do this stuff? -``` - -**16.** No es necesario escribir un comentario al final de un bloque que describa de qué se trataba. - -``` cpp -/// for -``` - -## Nombre {#names} - -**1.** Use letras minúsculas con guiones bajos en los nombres de variables y miembros de clase. - -``` cpp -size_t max_block_size; -``` - -**2.** Para los nombres de las funciones (métodos), use camelCase comenzando con una letra minúscula. - -``` cpp -std::string getName() const override { return "Memory"; } -``` - -**3.** Para los nombres de las clases (estructuras), use CamelCase comenzando con una letra mayúscula. Los prefijos distintos de I no se usan para interfaces. - -``` cpp -class StorageMemory : public IStorage -``` - -**4.** `using` se nombran de la misma manera que las clases, o con `_t` al final. - -**5.** Nombres de argumentos de tipo de plantilla: en casos simples, use `T`; `T`, `U`; `T1`, `T2`. - -Para casos más complejos, siga las reglas para los nombres de clase o agregue el prefijo `T`. - -``` cpp -template -struct AggregatedStatElement -``` - -**6.** Nombres de argumentos constantes de plantilla: siga las reglas para los nombres de variables o use `N` en casos simples. - -``` cpp -template -struct ExtractDomain -``` - -**7.** Para clases abstractas (interfaces) puede agregar el `I` prefijo. - -``` cpp -class IBlockInputStream -``` - -**8.** Si usa una variable localmente, puede usar el nombre corto. - -En todos los demás casos, use un nombre que describa el significado. - -``` cpp -bool info_successfully_loaded = false; -``` - -**9.** Nombres de `define`s y las constantes globales usan ALL_CAPS con guiones bajos. - -``` cpp -#define MAX_SRC_TABLE_NAMES_TO_STORE 1000 -``` - -**10.** Los nombres de archivo deben usar el mismo estilo que su contenido. - -Si un archivo contiene una sola clase, nombre el archivo de la misma manera que la clase (CamelCase). - -Si el archivo contiene una sola función, nombre el archivo de la misma manera que la función (camelCase). - -**11.** Si el nombre contiene una abreviatura, : - -- Para los nombres de variables, la abreviatura debe usar letras minúsculas `mysql_connection` (ni `mySQL_connection`). -- Para los nombres de clases y funciones, mantenga las letras mayúsculas en la abreviatura`MySQLConnection` (ni `MySqlConnection`). - -**12.** Los argumentos del constructor que se usan solo para inicializar los miembros de la clase deben nombrarse de la misma manera que los miembros de la clase, pero con un guión bajo al final. - -``` cpp -FileQueueProcessor( - const std::string & path_, - const std::string & prefix_, - std::shared_ptr handler_) - : path(path_), - prefix(prefix_), - handler(handler_), - log(&Logger::get("FileQueueProcessor")) -{ -} -``` - -El sufijo de subrayado se puede omitir si el argumento no se usa en el cuerpo del constructor. - -**13.** No hay diferencia en los nombres de las variables locales y los miembros de la clase (no se requieren prefijos). - -``` cpp -timer (not m_timer) -``` - -**14.** Para las constantes en un `enum`, usar CamelCase con una letra mayúscula. ALL_CAPS también es aceptable. Si el `enum` no es local, utilice un `enum class`. - -``` cpp -enum class CompressionMethod -{ - QuickLZ = 0, - LZ4 = 1, -}; -``` - -**15.** Todos los nombres deben estar en inglés. La transliteración de palabras rusas no está permitida. - - not Stroka - -**16.** Las abreviaturas son aceptables si son bien conocidas (cuando puede encontrar fácilmente el significado de la abreviatura en Wikipedia o en un motor de búsqueda). - - `AST`, `SQL`. - - Not `NVDH` (some random letters) - -Las palabras incompletas son aceptables si la versión abreviada es de uso común. - -También puede usar una abreviatura si el nombre completo se incluye junto a él en los comentarios. - -**17.** Los nombres de archivo con código fuente de C++ deben tener `.cpp` ampliación. Los archivos de encabezado deben tener `.h` ampliación. - -## Cómo escribir código {#how-to-write-code} - -**1.** Gestión de la memoria. - -Desasignación de memoria manual (`delete`) solo se puede usar en el código de la biblioteca. - -En el código de la biblioteca, el `delete` operador sólo se puede utilizar en destructores. - -En el código de la aplicación, la memoria debe ser liberada por el objeto que la posee. - -Ejemplos: - -- La forma más fácil es colocar un objeto en la pila o convertirlo en miembro de otra clase. -- Para una gran cantidad de objetos pequeños, use contenedores. -- Para la desasignación automática de un pequeño número de objetos que residen en el montón, use `shared_ptr/unique_ptr`. - -**2.** Gestión de recursos. - -Utilizar `RAII` y ver arriba. - -**3.** Manejo de errores. - -Utilice excepciones. En la mayoría de los casos, solo necesita lanzar una excepción y no necesita atraparla (debido a `RAII`). - -En las aplicaciones de procesamiento de datos fuera de línea, a menudo es aceptable no detectar excepciones. - -En los servidores que manejan las solicitudes de los usuarios, generalmente es suficiente detectar excepciones en el nivel superior del controlador de conexión. - -En las funciones de subproceso, debe capturar y mantener todas las excepciones para volver a lanzarlas en el subproceso principal después `join`. - -``` cpp -/// If there weren't any calculations yet, calculate the first block synchronously -if (!started) -{ - calculate(); - started = true; -} -else /// If calculations are already in progress, wait for the result - pool.wait(); - -if (exception) - exception->rethrow(); -``` - -Nunca oculte excepciones sin manejo. Nunca simplemente ponga ciegamente todas las excepciones para iniciar sesión. - -``` cpp -//Not correct -catch (...) {} -``` - -Si necesita ignorar algunas excepciones, hágalo solo para las específicas y vuelva a lanzar el resto. - -``` cpp -catch (const DB::Exception & e) -{ - if (e.code() == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION) - return nullptr; - else - throw; -} -``` - -Al usar funciones con códigos de respuesta o `errno`, siempre verifique el resultado y arroje una excepción en caso de error. - -``` cpp -if (0 != close(fd)) - throwFromErrno("Cannot close file " + file_name, ErrorCodes::CANNOT_CLOSE_FILE); -``` - -`Do not use assert`. - -**4.** Tipos de excepción. - -No es necesario utilizar una jerarquía de excepciones compleja en el código de la aplicación. El texto de excepción debe ser comprensible para un administrador del sistema. - -**5.** Lanzar excepciones de destructores. - -Esto no es recomendable, pero está permitido. - -Utilice las siguientes opciones: - -- Crear una función (`done()` o `finalize()`) que hará todo el trabajo de antemano que podría conducir a una excepción. Si se llamó a esa función, no debería haber excepciones en el destructor más adelante. -- Las tareas que son demasiado complejas (como enviar mensajes a través de la red) se pueden poner en un método separado al que el usuario de la clase tendrá que llamar antes de la destrucción. -- Si hay una excepción en el destructor, es mejor registrarla que ocultarla (si el registrador está disponible). -- En aplicaciones simples, es aceptable confiar en `std::terminate` (para los casos de `noexcept` de forma predeterminada en C ++ 11) para manejar excepciones. - -**6.** Bloques de código anónimos. - -Puede crear un bloque de código separado dentro de una sola función para hacer que ciertas variables sean locales, de modo que se llame a los destructores al salir del bloque. - -``` cpp -Block block = data.in->read(); - -{ - std::lock_guard lock(mutex); - data.ready = true; - data.block = block; -} - -ready_any.set(); -``` - -**7.** Multithreading. - -En programas de procesamiento de datos fuera de línea: - -- Trate de obtener el mejor rendimiento posible en un solo núcleo de CPU. A continuación, puede paralelizar su código si es necesario. - -En aplicaciones de servidor: - -- Utilice el grupo de subprocesos para procesar solicitudes. En este punto, no hemos tenido ninguna tarea que requiera el cambio de contexto de espacio de usuario. - -La horquilla no se usa para la paralelización. - -**8.** Sincronización de hilos. - -A menudo es posible hacer que diferentes hilos usen diferentes celdas de memoria (incluso mejor: diferentes líneas de caché) y no usar ninguna sincronización de hilos (excepto `joinAll`). - -Si se requiere sincronización, en la mayoría de los casos, es suficiente usar mutex bajo `lock_guard`. - -En otros casos, use primitivas de sincronización del sistema. No utilice la espera ocupada. - -Las operaciones atómicas deben usarse solo en los casos más simples. - -No intente implementar estructuras de datos sin bloqueo a menos que sea su principal área de especialización. - -**9.** Punteros vs referencias. - -En la mayoría de los casos, prefiera referencias. - -**10.** Construir. - -Usar referencias constantes, punteros a constantes, `const_iterator`, y métodos const. - -Considerar `const` para ser predeterminado y usar no-`const` sólo cuando sea necesario. - -Al pasar variables por valor, usando `const` por lo general no tiene sentido. - -**11.** sin firmar. - -Utilizar `unsigned` si es necesario. - -**12.** Tipos numéricos. - -Utilice los tipos `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, y `Int64`, así como `size_t`, `ssize_t`, y `ptrdiff_t`. - -No use estos tipos para números: `signed/unsigned long`, `long long`, `short`, `signed/unsigned char`, `char`. - -**13.** Pasando argumentos. - -Pasar valores complejos por referencia (incluyendo `std::string`). - -Si una función captura la propiedad de un objeto creado en el montón, cree el tipo de argumento `shared_ptr` o `unique_ptr`. - -**14.** Valores devueltos. - -En la mayoría de los casos, sólo tiene que utilizar `return`. No escribir `return std::move(res)`. - -Si la función asigna un objeto en el montón y lo devuelve, use `shared_ptr` o `unique_ptr`. - -En casos excepcionales, es posible que deba devolver el valor a través de un argumento. En este caso, el argumento debe ser una referencia. - -``` cpp -using AggregateFunctionPtr = std::shared_ptr; - -/** Allows creating an aggregate function by its name. - */ -class AggregateFunctionFactory -{ -public: - AggregateFunctionFactory(); - AggregateFunctionPtr get(const String & name, const DataTypes & argument_types) const; -``` - -**15.** espacio de nombres. - -No hay necesidad de usar un `namespace` para el código de aplicación. - -Las bibliotecas pequeñas tampoco necesitan esto. - -Para bibliotecas medianas a grandes, coloque todo en un `namespace`. - -En la biblioteca `.h` archivo, se puede utilizar `namespace detail` para ocultar los detalles de implementación no necesarios para el código de la aplicación. - -En un `.cpp` archivo, puede usar un `static` o espacio de nombres anónimo para ocultar símbolos. - -Además, un `namespace` puede ser utilizado para un `enum` para evitar que los nombres correspondientes caigan en un `namespace` (pero es mejor usar un `enum class`). - -**16.** Inicialización diferida. - -Si se requieren argumentos para la inicialización, normalmente no debe escribir un constructor predeterminado. - -Si más adelante tendrá que retrasar la inicialización, puede agregar un constructor predeterminado que creará un objeto no válido. O, para un pequeño número de objetos, puede usar `shared_ptr/unique_ptr`. - -``` cpp -Loader(DB::Connection * connection_, const std::string & query, size_t max_block_size_); - -/// For deferred initialization -Loader() {} -``` - -**17.** Funciones virtuales. - -Si la clase no está destinada para uso polimórfico, no necesita hacer que las funciones sean virtuales. Esto también se aplica al destructor. - -**18.** Codificación. - -Usa UTF-8 en todas partes. Utilizar `std::string`y`char *`. No use `std::wstring`y`wchar_t`. - -**19.** Tala. - -Vea los ejemplos en todas partes del código. - -Antes de confirmar, elimine todo el registro de depuración y sin sentido, y cualquier otro tipo de salida de depuración. - -Se debe evitar el registro en ciclos, incluso en el nivel Trace. - -Los registros deben ser legibles en cualquier nivel de registro. - -El registro solo debe usarse en el código de la aplicación, en su mayor parte. - -Los mensajes de registro deben estar escritos en inglés. - -El registro debe ser preferiblemente comprensible para el administrador del sistema. - -No use blasfemias en el registro. - -Utilice la codificación UTF-8 en el registro. En casos excepcionales, puede usar caracteres que no sean ASCII en el registro. - -**20.** Entrada-salida. - -No utilice `iostreams` en ciclos internos que son críticos para el rendimiento de la aplicación (y nunca usan `stringstream`). - -Utilice el `DB/IO` biblioteca en su lugar. - -**21.** Fecha y hora. - -Ver el `DateLUT` biblioteca. - -**22.** incluir. - -Utilice siempre `#pragma once` en lugar de incluir guardias. - -**23.** utilizar. - -`using namespace` no se utiliza. Usted puede utilizar `using` con algo específico. Pero hazlo local dentro de una clase o función. - -**24.** No use `trailing return type` para funciones a menos que sea necesario. - -``` cpp -auto f() -> void -``` - -**25.** Declaración e inicialización de variables. - -``` cpp -//right way -std::string s = "Hello"; -std::string s{"Hello"}; - -//wrong way -auto s = std::string{"Hello"}; -``` - -**26.** Para funciones virtuales, escriba `virtual` en la clase base, pero escribe `override` en lugar de `virtual` en las clases descendientes. - -## Características no utilizadas de C ++ {#unused-features-of-c} - -**1.** La herencia virtual no se utiliza. - -**2.** Los especificadores de excepción de C ++ 03 no se usan. - -## Plataforma {#platform} - -**1.** Escribimos código para una plataforma específica. - -Pero en igualdad de condiciones, se prefiere el código multiplataforma o portátil. - -**2.** Idioma: C++20. - -**3.** Compilación: `gcc`. En este momento (agosto 2020), el código se compila utilizando la versión 9.3. (También se puede compilar usando `clang 8`.) - -Se utiliza la biblioteca estándar (`libc++`). - -**4.**OS: Linux Ubuntu, no más viejo que Precise. - -**5.**El código está escrito para la arquitectura de CPU x86_64. - -El conjunto de instrucciones de CPU es el conjunto mínimo admitido entre nuestros servidores. Actualmente, es SSE 4.2. - -**6.** Utilizar `-Wall -Wextra -Werror` flags de compilación. - -**7.** Use enlaces estáticos con todas las bibliotecas, excepto aquellas a las que son difíciles de conectar estáticamente (consulte la salida de la `ldd` comando). - -**8.** El código se desarrolla y se depura con la configuración de la versión. - -## Herramienta {#tools} - -**1.** KDevelop es un buen IDE. - -**2.** Para la depuración, use `gdb`, `valgrind` (`memcheck`), `strace`, `-fsanitize=...`, o `tcmalloc_minimal_debug`. - -**3.** Para crear perfiles, use `Linux Perf`, `valgrind` (`callgrind`), o `strace -cf`. - -**4.** Las fuentes están en Git. - -**5.** Usos de ensamblaje `CMake`. - -**6.** Los programas se lanzan usando `deb` paquete. - -**7.** Los compromisos a dominar no deben romper la compilación. - -Aunque solo las revisiones seleccionadas se consideran viables. - -**8.** Realice confirmaciones tan a menudo como sea posible, incluso si el código está parcialmente listo. - -Use ramas para este propósito. - -Si su código en el `master` branch todavía no se puede construir, excluirlo de la compilación antes de la `push`. Tendrá que terminarlo o eliminarlo dentro de unos días. - -**9.** Para cambios no triviales, use ramas y publíquelas en el servidor. - -**10.** El código no utilizado se elimina del repositorio. - -## Biblioteca {#libraries} - -**1.** Se utiliza la biblioteca estándar de C++20 (se permiten extensiones experimentales), así como `boost` y `Poco` marco. - -**2.** Si es necesario, puede usar cualquier biblioteca conocida disponible en el paquete del sistema operativo. - -Si ya hay una buena solución disponible, úsela, incluso si eso significa que debe instalar otra biblioteca. - -(Pero prepárese para eliminar las bibliotecas incorrectas del código.) - -**3.** Puede instalar una biblioteca que no esté en los paquetes, si los paquetes no tienen lo que necesita o tienen una versión obsoleta o el tipo de compilación incorrecto. - -**4.** Si la biblioteca es pequeña y no tiene su propio sistema de compilación complejo, coloque los archivos `contrib` carpeta. - -**5.** Siempre se da preferencia a las bibliotecas que ya están en uso. - -## Recomendaciones generales {#general-recommendations-1} - -**1.** Escribe el menor código posible. - -**2.** Pruebe la solución más simple. - -**3.** No escriba código hasta que sepa cómo va a funcionar y cómo funcionará el bucle interno. - -**4.** En los casos más simples, use `using` en lugar de clases o estructuras. - -**5.** Si es posible, no escriba constructores de copia, operadores de asignación, destructores (que no sean virtuales, si la clase contiene al menos una función virtual), mueva constructores o mueva operadores de asignación. En otras palabras, las funciones generadas por el compilador deben funcionar correctamente. Usted puede utilizar `default`. - -**6.** Se fomenta la simplificación del código. Reduzca el tamaño de su código siempre que sea posible. - -## Recomendaciones adicionales {#additional-recommendations} - -**1.** Especificar explícitamente `std::` para tipos de `stddef.h` - -no se recomienda. En otras palabras, recomendamos escribir `size_t` en su lugar `std::size_t` porque es más corto. - -Es aceptable agregar `std::`. - -**2.** Especificar explícitamente `std::` para funciones de la biblioteca C estándar - -no se recomienda. En otras palabras, escribir `memcpy` en lugar de `std::memcpy`. - -La razón es que hay funciones no estándar similares, tales como `memmem`. Utilizamos estas funciones en ocasiones. Estas funciones no existen en `namespace std`. - -Si usted escribe `std::memcpy` en lugar de `memcpy` en todas partes, entonces `memmem` sin `std::` se verá extraño. - -Sin embargo, todavía puedes usar `std::` si lo prefieres. - -**3.** Usar funciones de C cuando las mismas están disponibles en la biblioteca estándar de C ++. - -Esto es aceptable si es más eficiente. - -Por ejemplo, use `memcpy` en lugar de `std::copy` para copiar grandes trozos de memoria. - -**4.** Argumentos de función multilínea. - -Se permite cualquiera de los siguientes estilos de ajuste: - -``` cpp -function( - T1 x1, - T2 x2) -``` - -``` cpp -function( - size_t left, size_t right, - const & RangesInDataParts ranges, - size_t limit) -``` - -``` cpp -function(size_t left, size_t right, - const & RangesInDataParts ranges, - size_t limit) -``` - -``` cpp -function(size_t left, size_t right, - const & RangesInDataParts ranges, - size_t limit) -``` - -``` cpp -function( - size_t left, - size_t right, - const & RangesInDataParts ranges, - size_t limit) -``` - -[Artículo Original](https://clickhouse.tech/docs/en/development/style/) diff --git a/docs/es/development/tests.md b/docs/es/development/tests.md deleted file mode 120000 index c03d36c3916..00000000000 --- a/docs/es/development/tests.md +++ /dev/null @@ -1 +0,0 @@ -../../en/development/tests.md \ No newline at end of file diff --git a/docs/es/engines/database-engines/atomic.md b/docs/es/engines/database-engines/atomic.md deleted file mode 100644 index f019b94a00b..00000000000 --- a/docs/es/engines/database-engines/atomic.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -toc_priority: 32 -toc_title: Atomic ---- - - -# Atomic {#atomic} - -It is supports non-blocking `DROP` and `RENAME TABLE` queries and atomic `EXCHANGE TABLES t1 AND t2` queries. Atomic database engine is used by default. - -## Creating a Database {#creating-a-database} - -```sql -CREATE DATABASE test ENGINE = Atomic; -``` - -[Original article](https://clickhouse.tech/docs/en/engines/database_engines/atomic/) diff --git a/docs/es/engines/database-engines/index.md b/docs/es/engines/database-engines/index.md deleted file mode 100644 index 8784b9bd02b..00000000000 --- a/docs/es/engines/database-engines/index.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Motores de base de datos -toc_priority: 27 -toc_title: "Implantaci\xF3n" ---- - -# Motores de base de datos {#database-engines} - -Los motores de bases de datos le permiten trabajar con tablas. - -De forma predeterminada, ClickHouse utiliza su motor de base de datos nativa, que proporciona [motores de mesa](../../engines/table-engines/index.md) y una [Dialecto SQL](../../sql-reference/syntax.md). - -También puede utilizar los siguientes motores de base de datos: - -- [MySQL](mysql.md) - -- [Perezoso](lazy.md) - -[Artículo Original](https://clickhouse.tech/docs/en/database_engines/) diff --git a/docs/es/engines/database-engines/lazy.md b/docs/es/engines/database-engines/lazy.md deleted file mode 100644 index 0988c4cb395..00000000000 --- a/docs/es/engines/database-engines/lazy.md +++ /dev/null @@ -1,18 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: Perezoso ---- - -# Perezoso {#lazy} - -Mantiene las tablas en RAM solamente `expiration_time_in_seconds` segundos después del último acceso. Solo se puede usar con tablas \*Log. - -Está optimizado para almacenar muchas tablas pequeñas \* Log, para las cuales hay un largo intervalo de tiempo entre los accesos. - -## Creación de una base de datos {#creating-a-database} - - CREATE DATABASE testlazy ENGINE = Lazy(expiration_time_in_seconds); - -[Artículo Original](https://clickhouse.tech/docs/en/database_engines/lazy/) diff --git a/docs/es/engines/database-engines/mysql.md b/docs/es/engines/database-engines/mysql.md deleted file mode 100644 index 5f1dec97f35..00000000000 --- a/docs/es/engines/database-engines/mysql.md +++ /dev/null @@ -1,135 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 30 -toc_title: MySQL ---- - -# MySQL {#mysql} - -Permite conectarse a bases de datos en un servidor MySQL remoto y realizar `INSERT` y `SELECT` consultas para intercambiar datos entre ClickHouse y MySQL. - -El `MySQL` motor de base de datos traducir consultas al servidor MySQL para que pueda realizar operaciones tales como `SHOW TABLES` o `SHOW CREATE TABLE`. - -No puede realizar las siguientes consultas: - -- `RENAME` -- `CREATE TABLE` -- `ALTER` - -## Creación de una base de datos {#creating-a-database} - -``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] -ENGINE = MySQL('host:port', ['database' | database], 'user', 'password') -``` - -**Parámetros del motor** - -- `host:port` — MySQL server address. -- `database` — Remote database name. -- `user` — MySQL user. -- `password` — User password. - -## Soporte de tipos de datos {#data_types-support} - -| MySQL | Haga clic en Casa | -|----------------------------------|--------------------------------------------------------------| -| UNSIGNED TINYINT | [UInt8](../../sql-reference/data-types/int-uint.md) | -| TINYINT | [Int8](../../sql-reference/data-types/int-uint.md) | -| UNSIGNED SMALLINT | [UInt16](../../sql-reference/data-types/int-uint.md) | -| SMALLINT | [Int16](../../sql-reference/data-types/int-uint.md) | -| UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../../sql-reference/data-types/int-uint.md) | -| INT, MEDIUMINT | [Int32](../../sql-reference/data-types/int-uint.md) | -| UNSIGNED BIGINT | [UInt64](../../sql-reference/data-types/int-uint.md) | -| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) | -| FLOAT | [Float32](../../sql-reference/data-types/float.md) | -| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | -| DATE | [Fecha](../../sql-reference/data-types/date.md) | -| DATETIME, TIMESTAMP | [FechaHora](../../sql-reference/data-types/datetime.md) | -| BINARY | [Cadena fija](../../sql-reference/data-types/fixedstring.md) | - -Todos los demás tipos de datos MySQL se convierten en [Cadena](../../sql-reference/data-types/string.md). - -[NULL](../../sql-reference/data-types/nullable.md) se admite. - -## Ejemplos de uso {#examples-of-use} - -Tabla en MySQL: - -``` text -mysql> USE test; -Database changed - -mysql> CREATE TABLE `mysql_table` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `float` FLOAT NOT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into mysql_table (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from mysql_table; -+------+-----+ -| int_id | value | -+------+-----+ -| 1 | 2 | -+------+-----+ -1 row in set (0,00 sec) -``` - -Base de datos en ClickHouse, intercambiando datos con el servidor MySQL: - -``` sql -CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') -``` - -``` sql -SHOW DATABASES -``` - -``` text -┌─name─────┐ -│ default │ -│ mysql_db │ -│ system │ -└──────────┘ -``` - -``` sql -SHOW TABLES FROM mysql_db -``` - -``` text -┌─name─────────┐ -│ mysql_table │ -└──────────────┘ -``` - -``` sql -SELECT * FROM mysql_db.mysql_table -``` - -``` text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -└────────┴───────┘ -``` - -``` sql -INSERT INTO mysql_db.mysql_table VALUES (3,4) -``` - -``` sql -SELECT * FROM mysql_db.mysql_table -``` - -``` text -┌─int_id─┬─value─┐ -│ 1 │ 2 │ -│ 3 │ 4 │ -└────────┴───────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/database_engines/mysql/) diff --git a/docs/es/engines/index.md b/docs/es/engines/index.md deleted file mode 100644 index 03e4426dd8d..00000000000 --- a/docs/es/engines/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Motor -toc_priority: 25 ---- - - diff --git a/docs/es/engines/table-engines/index.md b/docs/es/engines/table-engines/index.md deleted file mode 100644 index 7be315e3ee3..00000000000 --- a/docs/es/engines/table-engines/index.md +++ /dev/null @@ -1,85 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Motores de mesa -toc_priority: 26 -toc_title: "Implantaci\xF3n" ---- - -# Motores de mesa {#table_engines} - -El motor de tabla (tipo de tabla) determina: - -- Cómo y dónde se almacenan los datos, dónde escribirlos y dónde leerlos. -- Qué consultas son compatibles y cómo. -- Acceso a datos simultáneos. -- Uso de índices, si está presente. -- Si es posible la ejecución de solicitudes multiproceso. -- Parámetros de replicación de datos. - -## Familias de motores {#engine-families} - -### Método de codificación de datos: {#mergetree} - -Los motores de mesa más universales y funcionales para tareas de alta carga. La propiedad compartida por estos motores es la inserción rápida de datos con el posterior procesamiento de datos en segundo plano. `MergeTree` Los motores familiares admiten la replicación de datos (con [Replicado\*](mergetree-family/replication.md#table_engines-replication) versiones de motores), particionamiento y otras características no admitidas en otros motores. - -Motores en la familia: - -- [Método de codificación de datos:](mergetree-family/mergetree.md#mergetree) -- [ReplacingMergeTree](mergetree-family/replacingmergetree.md#replacingmergetree) -- [SummingMergeTree](mergetree-family/summingmergetree.md#summingmergetree) -- [AgregaciónMergeTree](mergetree-family/aggregatingmergetree.md#aggregatingmergetree) -- [ColapsarMergeTree](mergetree-family/collapsingmergetree.md#table_engine-collapsingmergetree) -- [VersionedCollapsingMergeTree](mergetree-family/versionedcollapsingmergetree.md#versionedcollapsingmergetree) -- [GraphiteMergeTree](mergetree-family/graphitemergetree.md#graphitemergetree) - -### Registro {#log} - -Ligero [motor](log-family/index.md) con funcionalidad mínima. Son los más efectivos cuando necesita escribir rápidamente muchas tablas pequeñas (hasta aproximadamente 1 millón de filas) y leerlas más tarde como un todo. - -Motores en la familia: - -- [TinyLog](log-family/tinylog.md#tinylog) -- [StripeLog](log-family/stripelog.md#stripelog) -- [Registro](log-family/log.md#log) - -### Motores de integración {#integration-engines} - -Motores para comunicarse con otros sistemas de almacenamiento y procesamiento de datos. - -Motores en la familia: - -- [Kafka](integrations/kafka.md#kafka) -- [MySQL](integrations/mysql.md#mysql) -- [ODBC](integrations/odbc.md#table-engine-odbc) -- [JDBC](integrations/jdbc.md#table-engine-jdbc) -- [HDFS](integrations/hdfs.md#hdfs) - -### Motores especiales {#special-engines} - -Motores en la familia: - -- [Distribuido](special/distributed.md#distributed) -- [Método de codificación de datos:](special/materializedview.md#materializedview) -- [Diccionario](special/dictionary.md#dictionary) -- \[Fusión\](special/merge.md#merge -- [File](special/file.md#file) -- [Nulo](special/null.md#null) -- [Establecer](special/set.md#set) -- [Unir](special/join.md#join) -- [URL](special/url.md#table_engines-url) -- [Vista](special/view.md#table_engines-view) -- [Memoria](special/memory.md#memory) -- [Búfer](special/buffer.md#buffer) - -## Virtual Columnas {#table_engines-virtual_columns} - -La columna virtual es un atributo de motor de tabla integral que se define en el código fuente del motor. - -No debe especificar columnas virtuales en el `CREATE TABLE` consulta y no puedes verlos en `SHOW CREATE TABLE` y `DESCRIBE TABLE` resultados de la consulta. Las columnas virtuales también son de solo lectura, por lo que no puede insertar datos en columnas virtuales. - -Para seleccionar datos de una columna virtual, debe especificar su nombre en el `SELECT` consulta. `SELECT *` no devuelve valores de columnas virtuales. - -Si crea una tabla con una columna que tiene el mismo nombre que una de las columnas virtuales de la tabla, la columna virtual se vuelve inaccesible. No recomendamos hacer esto. Para ayudar a evitar conflictos, los nombres de columna virtual suelen tener el prefijo de un guión bajo. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/) diff --git a/docs/es/engines/table-engines/integrations/hdfs.md b/docs/es/engines/table-engines/integrations/hdfs.md deleted file mode 100644 index 5e0211660f5..00000000000 --- a/docs/es/engines/table-engines/integrations/hdfs.md +++ /dev/null @@ -1,123 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: HDFS ---- - -# HDFS {#table_engines-hdfs} - -Este motor proporciona integración con [Acerca de nosotros](https://en.wikipedia.org/wiki/Apache_Hadoop) permitiendo gestionar datos sobre [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)a través de ClickHouse. Este motor es similar -a la [File](../special/file.md#table_engines-file) y [URL](../special/url.md#table_engines-url) motores, pero proporciona características específicas de Hadoop. - -## Uso {#usage} - -``` sql -ENGINE = HDFS(URI, format) -``` - -El `URI` El parámetro es el URI del archivo completo en HDFS. -El `format` parámetro especifica uno de los formatos de archivo disponibles. Realizar -`SELECT` consultas, el formato debe ser compatible para la entrada, y para realizar -`INSERT` queries – for output. The available formats are listed in the -[Formato](../../../interfaces/formats.md#formats) apartado. -La parte de la ruta de `URI` puede contener globs. En este caso, la tabla sería de solo lectura. - -**Ejemplo:** - -**1.** Configurar el `hdfs_engine_table` tabla: - -``` sql -CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') -``` - -**2.** Llenar archivo: - -``` sql -INSERT INTO hdfs_engine_table VALUES ('one', 1), ('two', 2), ('three', 3) -``` - -**3.** Consultar los datos: - -``` sql -SELECT * FROM hdfs_engine_table LIMIT 2 -``` - -``` text -┌─name─┬─value─┐ -│ one │ 1 │ -│ two │ 2 │ -└──────┴───────┘ -``` - -## Detalles de implementación {#implementation-details} - -- Las lecturas y escrituras pueden ser paralelas -- No soportado: - - `ALTER` y `SELECT...SAMPLE` operación. - - Índices. - - Replicación. - -**Globs en el camino** - -Múltiples componentes de ruta de acceso pueden tener globs. Para ser procesado, el archivo debe existir y coincidir con todo el patrón de ruta. Listado de archivos determina durante `SELECT` (no en `CREATE` momento). - -- `*` — Substitutes any number of any characters except `/` incluyendo cadena vacía. -- `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. -- `{N..M}` — Substitutes any number in range from N to M including both borders. - -Construcciones con `{}` son similares a la [remoto](../../../sql-reference/table-functions/remote.md) función de la tabla. - -**Ejemplo** - -1. Supongamos que tenemos varios archivos en formato TSV con los siguientes URI en HDFS: - -- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ - -1. Hay varias maneras de hacer una tabla que consta de los seis archivos: - - - -``` sql -CREATE TABLE table_with_range (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/some_file_{1..3}', 'TSV') -``` - -Otra forma: - -``` sql -CREATE TABLE table_with_question_mark (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/some_file_?', 'TSV') -``` - -La tabla consta de todos los archivos en ambos directorios (todos los archivos deben satisfacer el formato y el esquema descritos en la consulta): - -``` sql -CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV') -``` - -!!! warning "Advertencia" - Si la lista de archivos contiene rangos de números con ceros a la izquierda, use la construcción con llaves para cada dígito por separado o use `?`. - -**Ejemplo** - -Crear tabla con archivos llamados `file000`, `file001`, … , `file999`: - -``` sql -CREARE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV') -``` - -## Virtual Columnas {#virtual-columns} - -- `_path` — Path to the file. -- `_file` — Name of the file. - -**Ver también** - -- [Virtual columnas](../index.md#table_engines-virtual_columns) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/hdfs/) diff --git a/docs/es/engines/table-engines/integrations/index.md b/docs/es/engines/table-engines/integrations/index.md deleted file mode 100644 index e57aaf88744..00000000000 --- a/docs/es/engines/table-engines/integrations/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Integraci\xF3n" -toc_priority: 30 ---- - - diff --git a/docs/es/engines/table-engines/integrations/jdbc.md b/docs/es/engines/table-engines/integrations/jdbc.md deleted file mode 100644 index fd3450cef7c..00000000000 --- a/docs/es/engines/table-engines/integrations/jdbc.md +++ /dev/null @@ -1,90 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 34 -toc_title: JDBC ---- - -# JDBC {#table-engine-jdbc} - -Permite que ClickHouse se conecte a bases de datos externas a través de [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity). - -Para implementar la conexión JDBC, ClickHouse utiliza el programa independiente [Sistema abierto.](https://github.com/alex-krash/clickhouse-jdbc-bridge) que debería ejecutarse como un demonio. - -Este motor soporta el [NULL](../../../sql-reference/data-types/nullable.md) tipo de datos. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name -( - columns list... -) -ENGINE = JDBC(dbms_uri, external_database, external_table) -``` - -**Parámetros del motor** - -- `dbms_uri` — URI of an external DBMS. - - Formato: `jdbc:://:/?user=&password=`. - Ejemplo para MySQL: `jdbc:mysql://localhost:3306/?user=root&password=root`. - -- `external_database` — Database in an external DBMS. - -- `external_table` — Name of the table in `external_database`. - -## Ejemplo de uso {#usage-example} - -Crear una tabla en el servidor MySQL conectándose directamente con su cliente de consola: - -``` text -mysql> CREATE TABLE `test`.`test` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, - -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from test; -+------+----------+-----+----------+ -| int_id | int_nullable | float | float_nullable | -+------+----------+-----+----------+ -| 1 | NULL | 2 | NULL | -+------+----------+-----+----------+ -1 row in set (0,00 sec) -``` - -Creación de una tabla en el servidor ClickHouse y selección de datos de ella: - -``` sql -CREATE TABLE jdbc_table -( - `int_id` Int32, - `int_nullable` Nullable(Int32), - `float` Float32, - `float_nullable` Nullable(Float32) -) -ENGINE JDBC('jdbc:mysql://localhost:3306/?user=root&password=root', 'test', 'test') -``` - -``` sql -SELECT * -FROM jdbc_table -``` - -``` text -┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐ -│ 1 │ ᴺᵁᴸᴸ │ 2 │ ᴺᵁᴸᴸ │ -└────────┴──────────────┴───────┴────────────────┘ -``` - -## Ver también {#see-also} - -- [Función de la tabla de JDBC](../../../sql-reference/table-functions/jdbc.md). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/jdbc/) diff --git a/docs/es/engines/table-engines/integrations/kafka.md b/docs/es/engines/table-engines/integrations/kafka.md deleted file mode 100644 index 54250aae82a..00000000000 --- a/docs/es/engines/table-engines/integrations/kafka.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 32 -toc_title: Kafka ---- - -# Kafka {#kafka} - -Este motor funciona con [Acerca de nosotros](http://kafka.apache.org/). - -Kafka te permite: - -- Publicar o suscribirse a flujos de datos. -- Organice el almacenamiento tolerante a fallos. -- Secuencias de proceso a medida que estén disponibles. - -## Creación de una tabla {#table_engine-kafka-creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = Kafka() -SETTINGS - kafka_broker_list = 'host:port', - kafka_topic_list = 'topic1,topic2,...', - kafka_group_name = 'group_name', - kafka_format = 'data_format'[,] - [kafka_row_delimiter = 'delimiter_symbol',] - [kafka_schema = '',] - [kafka_num_consumers = N,] - [kafka_max_block_size = 0,] - [kafka_skip_broken_messages = N,] - [kafka_commit_every_batch = 0] -``` - -Parámetros requeridos: - -- `kafka_broker_list` – A comma-separated list of brokers (for example, `localhost:9092`). -- `kafka_topic_list` – A list of Kafka topics. -- `kafka_group_name` – A group of Kafka consumers. Reading margins are tracked for each group separately. If you don't want messages to be duplicated in the cluster, use the same group name everywhere. -- `kafka_format` – Message format. Uses the same notation as the SQL `FORMAT` función, tal como `JSONEachRow`. Para obtener más información, consulte [Formato](../../../interfaces/formats.md) apartado. - -Parámetros opcionales: - -- `kafka_row_delimiter` – Delimiter character, which ends the message. -- `kafka_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap'n Proto](https://capnproto.org/) requiere la ruta de acceso al archivo de esquema y el nombre de la raíz `schema.capnp:Message` objeto. -- `kafka_num_consumers` – The number of consumers per table. Default: `1`. Especifique más consumidores si el rendimiento de un consumidor es insuficiente. El número total de consumidores no debe exceder el número de particiones en el tema, ya que solo se puede asignar un consumidor por partición. -- `kafka_max_block_size` - El tamaño máximo de lote (en mensajes) para la encuesta (predeterminado: `max_block_size`). -- `kafka_skip_broken_messages` – Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. Si `kafka_skip_broken_messages = N` entonces el motor salta *N* Mensajes de Kafka que no se pueden analizar (un mensaje es igual a una fila de datos). -- `kafka_commit_every_batch` - Confirmar cada lote consumido y manejado en lugar de una única confirmación después de escribir un bloque completo (predeterminado: `0`). - -Ejemplos: - -``` sql - CREATE TABLE queue ( - timestamp UInt64, - level String, - message String - ) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'JSONEachRow'); - - SELECT * FROM queue LIMIT 5; - - CREATE TABLE queue2 ( - timestamp UInt64, - level String, - message String - ) ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', - kafka_topic_list = 'topic', - kafka_group_name = 'group1', - kafka_format = 'JSONEachRow', - kafka_num_consumers = 4; - - CREATE TABLE queue2 ( - timestamp UInt64, - level String, - message String - ) ENGINE = Kafka('localhost:9092', 'topic', 'group1') - SETTINGS kafka_format = 'JSONEachRow', - kafka_num_consumers = 4; -``` - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No utilice este método en nuevos proyectos. Si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format - [, kafka_row_delimiter, kafka_schema, kafka_num_consumers, kafka_skip_broken_messages]) -``` - -
- -## Descripci {#description} - -Los mensajes entregados se realizan un seguimiento automático, por lo que cada mensaje de un grupo solo se cuenta una vez. Si desea obtener los datos dos veces, cree una copia de la tabla con otro nombre de grupo. - -Los grupos son flexibles y se sincronizan en el clúster. Por ejemplo, si tiene 10 temas y 5 copias de una tabla en un clúster, cada copia obtiene 2 temas. Si el número de copias cambia, los temas se redistribuyen automáticamente entre las copias. Lea más sobre esto en http://kafka.apache.org/intro . - -`SELECT` no es particularmente útil para leer mensajes (excepto para la depuración), ya que cada mensaje se puede leer solo una vez. Es más práctico crear subprocesos en tiempo real utilizando vistas materializadas. Para hacer esto: - -1. Use el motor para crear un consumidor de Kafka y considérelo como un flujo de datos. -2. Crea una tabla con la estructura deseada. -3. Cree una vista materializada que convierta los datos del motor y los coloque en una tabla creada previamente. - -Cuando el `MATERIALIZED VIEW` se une al motor, comienza a recopilar datos en segundo plano. Esto le permite recibir continuamente mensajes de Kafka y convertirlos al formato requerido usando `SELECT`. -Una tabla kafka puede tener tantas vistas materializadas como desee, no leen datos de la tabla kafka directamente, sino que reciben nuevos registros (en bloques), de esta manera puede escribir en varias tablas con diferentes niveles de detalle (con agrupación - agregación y sin). - -Ejemplo: - -``` sql - CREATE TABLE queue ( - timestamp UInt64, - level String, - message String - ) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'JSONEachRow'); - - CREATE TABLE daily ( - day Date, - level String, - total UInt64 - ) ENGINE = SummingMergeTree(day, (day, level), 8192); - - CREATE MATERIALIZED VIEW consumer TO daily - AS SELECT toDate(toDateTime(timestamp)) AS day, level, count() as total - FROM queue GROUP BY day, level; - - SELECT level, sum(total) FROM daily GROUP BY level; -``` - -Para mejorar el rendimiento, los mensajes recibidos se agrupan en bloques del tamaño de [Max_insert_block_size](../../../operations/server-configuration-parameters/settings.md#settings-max_insert_block_size). Si el bloque no se formó dentro de [Nombre de la red inalámbrica (SSID):](../../../operations/server-configuration-parameters/settings.md) milisegundos, los datos se vaciarán a la tabla independientemente de la integridad del bloque. - -Para detener la recepción de datos de tema o cambiar la lógica de conversión, desconecte la vista materializada: - -``` sql - DETACH TABLE consumer; - ATTACH TABLE consumer; -``` - -Si desea cambiar la tabla de destino utilizando `ALTER`, recomendamos deshabilitar la vista de material para evitar discrepancias entre la tabla de destino y los datos de la vista. - -## Configuración {#configuration} - -Similar a GraphiteMergeTree, el motor Kafka admite una configuración extendida utilizando el archivo de configuración ClickHouse. Hay dos claves de configuración que puede usar: global (`kafka`) y a nivel de tema (`kafka_*`). La configuración global se aplica primero y, a continuación, se aplica la configuración de nivel de tema (si existe). - -``` xml - - - cgrp - smallest - - - - - 250 - 100000 - -``` - -Para obtener una lista de posibles opciones de configuración, consulte [referencia de configuración librdkafka](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Usa el guión bajo (`_`) en lugar de un punto en la configuración de ClickHouse. Por ejemplo, `check.crcs=true` será `true`. - -## Virtual Columnas {#virtual-columns} - -- `_topic` — Kafka topic. -- `_key` — Key of the message. -- `_offset` — Offset of the message. -- `_timestamp` — Timestamp of the message. -- `_partition` — Partition of Kafka topic. - -**Ver también** - -- [Virtual columnas](../index.md#table_engines-virtual_columns) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/kafka/) diff --git a/docs/es/engines/table-engines/integrations/mysql.md b/docs/es/engines/table-engines/integrations/mysql.md deleted file mode 100644 index 52799117255..00000000000 --- a/docs/es/engines/table-engines/integrations/mysql.md +++ /dev/null @@ -1,105 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 33 -toc_title: MySQL ---- - -# Mysql {#mysql} - -El motor MySQL le permite realizar `SELECT` consultas sobre datos almacenados en un servidor MySQL remoto. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], - ... -) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']); -``` - -Vea una descripción detallada del [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) consulta. - -La estructura de la tabla puede diferir de la estructura de la tabla MySQL original: - -- Los nombres de columna deben ser los mismos que en la tabla MySQL original, pero puede usar solo algunas de estas columnas y en cualquier orden. -- Los tipos de columna pueden diferir de los de la tabla MySQL original. ClickHouse intenta [elenco](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) valores a los tipos de datos ClickHouse. - -**Parámetros del motor** - -- `host:port` — MySQL server address. - -- `database` — Remote database name. - -- `table` — Remote table name. - -- `user` — MySQL user. - -- `password` — User password. - -- `replace_query` — Flag that converts `INSERT INTO` consultas a `REPLACE INTO`. Si `replace_query=1`, la consulta se sustituye. - -- `on_duplicate_clause` — The `ON DUPLICATE KEY on_duplicate_clause` expresión que se añade a la `INSERT` consulta. - - Ejemplo: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, donde `on_duplicate_clause` ser `UPDATE c2 = c2 + 1`. Ver el [Documentación de MySQL](https://dev.mysql.com/doc/refman/8.0/en/insert-on-duplicate.html) para encontrar qué `on_duplicate_clause` se puede utilizar con el `ON DUPLICATE KEY` clausula. - - Especificar `on_duplicate_clause` tienes que pasar `0` a la `replace_query` parámetro. Si pasa simultáneamente `replace_query = 1` y `on_duplicate_clause`, ClickHouse genera una excepción. - -Simple `WHERE` cláusulas tales como `=, !=, >, >=, <, <=` se ejecutan en el servidor MySQL. - -El resto de las condiciones y el `LIMIT` La restricción de muestreo se ejecuta en ClickHouse solo después de que finalice la consulta a MySQL. - -## Ejemplo de uso {#usage-example} - -Tabla en MySQL: - -``` text -mysql> CREATE TABLE `test`.`test` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, - -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from test; -+------+----------+-----+----------+ -| int_id | int_nullable | float | float_nullable | -+------+----------+-----+----------+ -| 1 | NULL | 2 | NULL | -+------+----------+-----+----------+ -1 row in set (0,00 sec) -``` - -Tabla en ClickHouse, recuperando datos de la tabla MySQL creada anteriormente: - -``` sql -CREATE TABLE mysql_table -( - `float_nullable` Nullable(Float32), - `int_id` Int32 -) -ENGINE = MySQL('localhost:3306', 'test', 'test', 'bayonet', '123') -``` - -``` sql -SELECT * FROM mysql_table -``` - -``` text -┌─float_nullable─┬─int_id─┐ -│ ᴺᵁᴸᴸ │ 1 │ -└────────────────┴────────┘ -``` - -## Ver también {#see-also} - -- [El ‘mysql’ función de la tabla](../../../sql-reference/table-functions/mysql.md) -- [Uso de MySQL como fuente de diccionario externo](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/mysql/) diff --git a/docs/es/engines/table-engines/integrations/odbc.md b/docs/es/engines/table-engines/integrations/odbc.md deleted file mode 100644 index 75c79484d61..00000000000 --- a/docs/es/engines/table-engines/integrations/odbc.md +++ /dev/null @@ -1,132 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 35 -toc_title: ODBC ---- - -# ODBC {#table-engine-odbc} - -Permite que ClickHouse se conecte a bases de datos externas a través de [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity). - -Para implementar con seguridad conexiones ODBC, ClickHouse usa un programa separado `clickhouse-odbc-bridge`. Si el controlador ODBC se carga directamente desde `clickhouse-server`, problemas de controlador pueden bloquear el servidor ClickHouse. ClickHouse se inicia automáticamente `clickhouse-odbc-bridge` cuando se requiere. El programa de puente ODBC se instala desde el mismo paquete que el `clickhouse-server`. - -Este motor soporta el [NULL](../../../sql-reference/data-types/nullable.md) tipo de datos. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1], - name2 [type2], - ... -) -ENGINE = ODBC(connection_settings, external_database, external_table) -``` - -Vea una descripción detallada del [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) consulta. - -La estructura de la tabla puede diferir de la estructura de la tabla de origen: - -- Los nombres de columna deben ser los mismos que en la tabla de origen, pero puede usar solo algunas de estas columnas y en cualquier orden. -- Los tipos de columna pueden diferir de los de la tabla de origen. ClickHouse intenta [elenco](../../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) valores a los tipos de datos ClickHouse. - -**Parámetros del motor** - -- `connection_settings` — Name of the section with connection settings in the `odbc.ini` file. -- `external_database` — Name of a database in an external DBMS. -- `external_table` — Name of a table in the `external_database`. - -## Ejemplo de uso {#usage-example} - -**Recuperación de datos de la instalación local de MySQL a través de ODBC** - -Este ejemplo se comprueba para Ubuntu Linux 18.04 y el servidor MySQL 5.7. - -Asegúrese de que unixODBC y MySQL Connector están instalados. - -De forma predeterminada (si se instala desde paquetes), ClickHouse comienza como usuario `clickhouse`. Por lo tanto, debe crear y configurar este usuario en el servidor MySQL. - -``` bash -$ sudo mysql -``` - -``` sql -mysql> CREATE USER 'clickhouse'@'localhost' IDENTIFIED BY 'clickhouse'; -mysql> GRANT ALL PRIVILEGES ON *.* TO 'clickhouse'@'clickhouse' WITH GRANT OPTION; -``` - -A continuación, configure la conexión en `/etc/odbc.ini`. - -``` bash -$ cat /etc/odbc.ini -[mysqlconn] -DRIVER = /usr/local/lib/libmyodbc5w.so -SERVER = 127.0.0.1 -PORT = 3306 -DATABASE = test -USERNAME = clickhouse -PASSWORD = clickhouse -``` - -Puede verificar la conexión usando el `isql` utilidad desde la instalación de unixODBC. - -``` bash -$ isql -v mysqlconn -+-------------------------+ -| Connected! | -| | -... -``` - -Tabla en MySQL: - -``` text -mysql> CREATE TABLE `test`.`test` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, - -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from test; -+------+----------+-----+----------+ -| int_id | int_nullable | float | float_nullable | -+------+----------+-----+----------+ -| 1 | NULL | 2 | NULL | -+------+----------+-----+----------+ -1 row in set (0,00 sec) -``` - -Tabla en ClickHouse, recuperando datos de la tabla MySQL: - -``` sql -CREATE TABLE odbc_t -( - `int_id` Int32, - `float_nullable` Nullable(Float32) -) -ENGINE = ODBC('DSN=mysqlconn', 'test', 'test') -``` - -``` sql -SELECT * FROM odbc_t -``` - -``` text -┌─int_id─┬─float_nullable─┐ -│ 1 │ ᴺᵁᴸᴸ │ -└────────┴────────────────┘ -``` - -## Ver también {#see-also} - -- [Diccionarios externos ODBC](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) -- [Tabla ODBC función](../../../sql-reference/table-functions/odbc.md) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/odbc/) diff --git a/docs/es/engines/table-engines/log-family/index.md b/docs/es/engines/table-engines/log-family/index.md deleted file mode 100644 index a7a3016f967..00000000000 --- a/docs/es/engines/table-engines/log-family/index.md +++ /dev/null @@ -1,47 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Familia de registro -toc_priority: 29 -toc_title: "Implantaci\xF3n" ---- - -# Familia del motor de registro {#log-engine-family} - -Estos motores fueron desarrollados para escenarios en los que necesita escribir rápidamente muchas tablas pequeñas (hasta aproximadamente 1 millón de filas) y leerlas más tarde en su conjunto. - -Motores de la familia: - -- [StripeLog](stripelog.md) -- [Registro](log.md) -- [TinyLog](tinylog.md) - -## Propiedades comunes {#common-properties} - -Motor: - -- Almacenar datos en un disco. - -- Agregue datos al final del archivo al escribir. - -- Bloqueos de soporte para el acceso a datos simultáneos. - - Durante `INSERT` consultas, la tabla está bloqueada y otras consultas para leer y escribir datos esperan a que la tabla se desbloquee. Si no hay consultas de escritura de datos, se puede realizar cualquier número de consultas de lectura de datos simultáneamente. - -- No apoyo [mutación](../../../sql-reference/statements/alter.md#alter-mutations) operación. - -- No admite índices. - - Esto significa que `SELECT` las consultas para rangos de datos no son eficientes. - -- No escriba datos atómicamente. - - Puede obtener una tabla con datos dañados si algo rompe la operación de escritura, por ejemplo, un cierre anormal del servidor. - -## Diferencia {#differences} - -El `TinyLog` es el más simple de la familia y proporciona la funcionalidad más pobre y la eficiencia más baja. El `TinyLog` el motor no admite la lectura de datos paralelos por varios hilos. Lee datos más lentamente que otros motores de la familia que admiten lectura paralela y utiliza casi tantos descriptores como los `Log` motor porque almacena cada columna en un archivo separado. Úselo en escenarios simples de baja carga. - -El `Log` y `StripeLog` Los motores admiten lectura de datos paralela. Al leer datos, ClickHouse usa múltiples hilos. Cada subproceso procesa un bloque de datos separado. El `Log` utiliza un archivo separado para cada columna de la tabla. `StripeLog` almacena todos los datos en un archivo. Como resultado, el `StripeLog` el motor utiliza menos descriptores en el sistema operativo, pero el `Log` proporciona una mayor eficiencia al leer datos. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/es/engines/table-engines/log-family/log.md b/docs/es/engines/table-engines/log-family/log.md deleted file mode 100644 index 1db374390e4..00000000000 --- a/docs/es/engines/table-engines/log-family/log.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 33 -toc_title: Registro ---- - -# Registro {#log} - -El motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](index.md) artículo. - -El registro difiere de [TinyLog](tinylog.md) en que un pequeño archivo de “marks” reside con los archivos de columna. Estas marcas se escriben en cada bloque de datos y contienen compensaciones que indican dónde comenzar a leer el archivo para omitir el número especificado de filas. Esto hace posible leer datos de tabla en múltiples hilos. -Para el acceso a datos simultáneos, las operaciones de lectura se pueden realizar simultáneamente, mientras que las operaciones de escritura bloquean las lecturas entre sí. -El motor de registro no admite índices. Del mismo modo, si la escritura en una tabla falla, la tabla se rompe y la lectura de ella devuelve un error. El motor de registro es adecuado para datos temporales, tablas de escritura única y para fines de prueba o demostración. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/log/) diff --git a/docs/es/engines/table-engines/log-family/stripelog.md b/docs/es/engines/table-engines/log-family/stripelog.md deleted file mode 100644 index 0965e9a987c..00000000000 --- a/docs/es/engines/table-engines/log-family/stripelog.md +++ /dev/null @@ -1,95 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 32 -toc_title: StripeLog ---- - -# Lista de Stripelog {#stripelog} - -Este motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](index.md) artículo. - -Utilice este motor en escenarios en los que necesite escribir muchas tablas con una pequeña cantidad de datos (menos de 1 millón de filas). - -## Creación de una tabla {#table_engines-stripelog-creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - column1_name [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - column2_name [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = StripeLog -``` - -Vea la descripción detallada del [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) consulta. - -## Escribir los datos {#table_engines-stripelog-writing-the-data} - -El `StripeLog` el motor almacena todas las columnas en un archivo. Para cada `INSERT` consulta, ClickHouse agrega el bloque de datos al final de un archivo de tabla, escribiendo columnas una por una. - -Para cada tabla, ClickHouse escribe los archivos: - -- `data.bin` — Data file. -- `index.mrk` — File with marks. Marks contain offsets for each column of each data block inserted. - -El `StripeLog` el motor no soporta el `ALTER UPDATE` y `ALTER DELETE` operación. - -## Lectura de los datos {#table_engines-stripelog-reading-the-data} - -El archivo con marcas permite ClickHouse paralelizar la lectura de datos. Esto significa que un `SELECT` query devuelve filas en un orden impredecible. Utilice el `ORDER BY` cláusula para ordenar filas. - -## Ejemplo de uso {#table_engines-stripelog-example-of-use} - -Creación de una tabla: - -``` sql -CREATE TABLE stripe_log_table -( - timestamp DateTime, - message_type String, - message String -) -ENGINE = StripeLog -``` - -Insertar datos: - -``` sql -INSERT INTO stripe_log_table VALUES (now(),'REGULAR','The first regular message') -INSERT INTO stripe_log_table VALUES (now(),'REGULAR','The second regular message'),(now(),'WARNING','The first warning message') -``` - -Se utilizaron dos `INSERT` consultas para crear dos bloques de datos dentro del `data.bin` file. - -ClickHouse usa múltiples subprocesos al seleccionar datos. Cada subproceso lee un bloque de datos separado y devuelve las filas resultantes de forma independiente a medida que termina. Como resultado, el orden de los bloques de filas en la salida no coincide con el orden de los mismos bloques en la entrada en la mayoría de los casos. Por ejemplo: - -``` sql -SELECT * FROM stripe_log_table -``` - -``` text -┌───────────timestamp─┬─message_type─┬─message────────────────────┐ -│ 2019-01-18 14:27:32 │ REGULAR │ The second regular message │ -│ 2019-01-18 14:34:53 │ WARNING │ The first warning message │ -└─────────────────────┴──────────────┴────────────────────────────┘ -┌───────────timestamp─┬─message_type─┬─message───────────────────┐ -│ 2019-01-18 14:23:43 │ REGULAR │ The first regular message │ -└─────────────────────┴──────────────┴───────────────────────────┘ -``` - -Ordenación de los resultados (orden ascendente por defecto): - -``` sql -SELECT * FROM stripe_log_table ORDER BY timestamp -``` - -``` text -┌───────────timestamp─┬─message_type─┬─message────────────────────┐ -│ 2019-01-18 14:23:43 │ REGULAR │ The first regular message │ -│ 2019-01-18 14:27:32 │ REGULAR │ The second regular message │ -│ 2019-01-18 14:34:53 │ WARNING │ The first warning message │ -└─────────────────────┴──────────────┴────────────────────────────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/stripelog/) diff --git a/docs/es/engines/table-engines/log-family/tinylog.md b/docs/es/engines/table-engines/log-family/tinylog.md deleted file mode 100644 index a2cbf7257b6..00000000000 --- a/docs/es/engines/table-engines/log-family/tinylog.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 34 -toc_title: TinyLog ---- - -# TinyLog {#tinylog} - -El motor pertenece a la familia de motores de registro. Ver [Familia del motor de registro](index.md) para las propiedades comunes de los motores de registro y sus diferencias. - -Este motor de tablas se usa normalmente con el método write-once: escribir datos una vez, luego leerlos tantas veces como sea necesario. Por ejemplo, puede usar `TinyLog`-type tablas para datos intermedios que se procesan en pequeños lotes. Tenga en cuenta que el almacenamiento de datos en un gran número de tablas pequeñas es ineficiente. - -Las consultas se ejecutan en una sola secuencia. En otras palabras, este motor está diseñado para tablas relativamente pequeñas (hasta aproximadamente 1,000,000 filas). Tiene sentido usar este motor de tablas si tiene muchas tablas pequeñas, ya que es más simple que el [Registro](log.md) motor (menos archivos necesitan ser abiertos). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/tinylog/) diff --git a/docs/es/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/es/engines/table-engines/mergetree-family/aggregatingmergetree.md deleted file mode 100644 index 2aedfbd2317..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ /dev/null @@ -1,105 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 35 -toc_title: "Agregaci\xF3nMergeTree" ---- - -# Aggregatingmergetree {#aggregatingmergetree} - -El motor hereda de [Método de codificación de datos:](mergetree.md#table_engines-mergetree), alterando la lógica para la fusión de partes de datos. ClickHouse reemplaza todas las filas con la misma clave principal (o más exactamente, con la misma [clave de clasificación](mergetree.md)) con una sola fila (dentro de una parte de datos) que almacena una combinación de estados de funciones agregadas. - -Usted puede utilizar `AggregatingMergeTree` tablas para la agregación de datos incrementales, incluidas las vistas materializadas agregadas. - -El motor procesa todas las columnas con los siguientes tipos: - -- [AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md) -- [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md) - -Es apropiado usar `AggregatingMergeTree` si reduce el número de filas por pedidos. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = AggregatingMergeTree() -[PARTITION BY expr] -[ORDER BY expr] -[SAMPLE BY expr] -[TTL expr] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros de solicitud, consulte [descripción de la solicitud](../../../sql-reference/statements/create.md). - -**Cláusulas de consulta** - -Al crear un `AggregatingMergeTree` mesa de la misma [clausula](mergetree.md) se requieren, como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No use este método en proyectos nuevos y, si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] AggregatingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity) -``` - -Todos los parámetros tienen el mismo significado que en `MergeTree`. -
- -## SELECCIONAR e INSERTAR {#select-and-insert} - -Para insertar datos, utilice [INSERT SELECT](../../../sql-reference/statements/insert-into.md) consulta con funciones agregadas -State-. -Al seleccionar datos de `AggregatingMergeTree` mesa, uso `GROUP BY` cláusula y las mismas funciones agregadas que al insertar datos, pero usando `-Merge` sufijo. - -En los resultados de `SELECT` consulta, los valores de `AggregateFunction` tipo tiene representación binaria específica de la implementación para todos los formatos de salida de ClickHouse. Si volcar datos en, por ejemplo, `TabSeparated` formato con `SELECT` consulta entonces este volcado se puede cargar de nuevo usando `INSERT` consulta. - -## Ejemplo de una vista materializada agregada {#example-of-an-aggregated-materialized-view} - -`AggregatingMergeTree` vista materializada que mira el `test.visits` tabla: - -``` sql -CREATE MATERIALIZED VIEW test.basic -ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) -AS SELECT - CounterID, - StartDate, - sumState(Sign) AS Visits, - uniqState(UserID) AS Users -FROM test.visits -GROUP BY CounterID, StartDate; -``` - -Insertar datos en el `test.visits` tabla. - -``` sql -INSERT INTO test.visits ... -``` - -Los datos se insertan tanto en la tabla como en la vista `test.basic` que realizará la agregación. - -Para obtener los datos agregados, necesitamos ejecutar una consulta como `SELECT ... GROUP BY ...` de la vista `test.basic`: - -``` sql -SELECT - StartDate, - sumMerge(Visits) AS Visits, - uniqMerge(Users) AS Users -FROM test.basic -GROUP BY StartDate -ORDER BY StartDate; -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/aggregatingmergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/es/engines/table-engines/mergetree-family/collapsingmergetree.md deleted file mode 100644 index 027d5c2adf7..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/collapsingmergetree.md +++ /dev/null @@ -1,306 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: ColapsarMergeTree ---- - -# ColapsarMergeTree {#table_engine-collapsingmergetree} - -El motor hereda de [Método de codificación de datos:](mergetree.md) y agrega la lógica de las filas que colapsan al algoritmo de fusión de partes de datos. - -`CollapsingMergeTree` elimina de forma asincrónica (colapsa) pares de filas si todos los campos de una clave de ordenación (`ORDER BY`) son equivalentes excepto el campo particular `Sign` que puede tener `1` y `-1` valor. Las filas sin un par se mantienen. Para más detalles, consulte el [Derrumbar](#table_engine-collapsingmergetree-collapsing) sección del documento. - -El motor puede reducir significativamente el volumen de almacenamiento y aumentar la eficiencia de `SELECT` consulta como consecuencia. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = CollapsingMergeTree(sign) -[PARTITION BY expr] -[ORDER BY expr] -[SAMPLE BY expr] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros de consulta, consulte [descripción de la consulta](../../../sql-reference/statements/create.md). - -**CollapsingMergeTree Parámetros** - -- `sign` — Name of the column with the type of row: `1` es una “state” fila, `-1` es una “cancel” fila. - - Column data type — `Int8`. - -**Cláusulas de consulta** - -Al crear un `CollapsingMergeTree` mesa, la misma [cláusulas de consulta](mergetree.md#table_engine-mergetree-creating-a-table) se requieren, como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No use este método en proyectos nuevos y, si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] CollapsingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, sign) -``` - -Todos los parámetros excepto `sign` el mismo significado que en `MergeTree`. - -- `sign` — Name of the column with the type of row: `1` — “state” fila, `-1` — “cancel” fila. - - Column Data Type — `Int8`. - -
- -## Derrumbar {#table_engine-collapsingmergetree-collapsing} - -### Datos {#data} - -Considere la situación en la que necesita guardar datos que cambian continuamente para algún objeto. Parece lógico tener una fila para un objeto y actualizarla en cualquier cambio, pero la operación de actualización es costosa y lenta para DBMS porque requiere la reescritura de los datos en el almacenamiento. Si necesita escribir datos rápidamente, la actualización no es aceptable, pero puede escribir los cambios de un objeto secuencialmente de la siguiente manera. - -Utilice la columna en particular `Sign`. Si `Sign = 1` significa que la fila es un estado de un objeto, llamémoslo “state” fila. Si `Sign = -1` significa la cancelación del estado de un objeto con los mismos atributos, llamémoslo “cancel” fila. - -Por ejemplo, queremos calcular cuántas páginas revisaron los usuarios en algún sitio y cuánto tiempo estuvieron allí. En algún momento escribimos la siguiente fila con el estado de la actividad del usuario: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -En algún momento después registramos el cambio de actividad del usuario y lo escribimos con las siguientes dos filas. - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -La primera fila cancela el estado anterior del objeto (usuario). Debe copiar los campos clave de ordenación del estado cancelado exceptuando `Sign`. - -La segunda fila contiene el estado actual. - -Como solo necesitamos el último estado de actividad del usuario, las filas - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -se puede eliminar colapsando el estado no válido (antiguo) de un objeto. `CollapsingMergeTree` hace esto mientras se fusionan las partes de datos. - -Por qué necesitamos 2 filas para cada cambio leído en el [Algoritmo](#table_engine-collapsingmergetree-collapsing-algorithm) apartado. - -**Propiedades peculiares de tal enfoque** - -1. El programa que escribe los datos debe recordar el estado de un objeto para poder cancelarlo. “Cancel” debe contener copias de los campos de clave de ordenación “state” y lo opuesto `Sign`. Aumenta el tamaño inicial de almacenamiento, pero permite escribir los datos rápidamente. -2. Las matrices de largo crecimiento en columnas reducen la eficiencia del motor debido a la carga para escribir. Los datos más sencillos, mayor será la eficiencia. -3. El `SELECT` Los resultados dependen en gran medida de la consistencia del historial de cambios de objetos. Sea preciso al preparar los datos para insertarlos. Puede obtener resultados impredecibles en datos incoherentes, por ejemplo, valores negativos para métricas no negativas, como la profundidad de la sesión. - -### Algoritmo {#table_engine-collapsingmergetree-collapsing-algorithm} - -Cuando ClickHouse combina partes de datos, cada grupo de filas consecutivas tiene la misma clave de ordenación (`ORDER BY`) se reduce a no más de dos filas, una con `Sign = 1` (“state” fila) y otro con `Sign = -1` (“cancel” fila). En otras palabras, las entradas colapsan. - -Para cada parte de datos resultante, ClickHouse guarda: - -1. El primero “cancel” y el último “state” si el número de “state” y “cancel” y la última fila es una “state” fila. -2. El último “state” fila, si hay más “state” filas que “cancel” filas. -3. El primero “cancel” fila, si hay más “cancel” filas que “state” filas. -4. Ninguna de las filas, en todos los demás casos. - -También cuando hay al menos 2 más “state” filas que “cancel” filas, o al menos 2 más “cancel” filas entonces “state” fila, la fusión continúa, pero ClickHouse trata esta situación como un error lógico y la registra en el registro del servidor. Este error puede producirse si se insertan los mismos datos más de una vez. - -Por lo tanto, el colapso no debe cambiar los resultados del cálculo de las estadísticas. -Los cambios colapsaron gradualmente para que al final solo quedara el último estado de casi todos los objetos. - -El `Sign` se requiere porque el algoritmo de fusión no garantiza que todas las filas con la misma clave de clasificación estén en la misma parte de datos resultante e incluso en el mismo servidor físico. Proceso de ClickHouse `SELECT` consultas con múltiples hilos, y no puede predecir el orden de las filas en el resultado. La agregación es necesaria si hay una necesidad de obtener completamente “collapsed” datos de `CollapsingMergeTree` tabla. - -Para finalizar el colapso, escriba una consulta con `GROUP BY` cláusula y funciones agregadas que representan el signo. Por ejemplo, para calcular la cantidad, use `sum(Sign)` en lugar de `count()`. Para calcular la suma de algo, use `sum(Sign * x)` en lugar de `sum(x)` y así sucesivamente, y también añadir `HAVING sum(Sign) > 0`. - -Los agregados `count`, `sum` y `avg` podría calcularse de esta manera. El agregado `uniq` podría calcularse si un objeto tiene al menos un estado no colapsado. Los agregados `min` y `max` no se pudo calcular porque `CollapsingMergeTree` no guarda el historial de valores de los estados colapsados. - -Si necesita extraer datos sin agregación (por ejemplo, para comprobar si hay filas presentes cuyos valores más recientes coinciden con ciertas condiciones), puede utilizar el `FINAL` modificador para el `FROM` clausula. Este enfoque es significativamente menos eficiente. - -## Ejemplo de uso {#example-of-use} - -Datos de ejemplo: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -Creación de la tabla: - -``` sql -CREATE TABLE UAct -( - UserID UInt64, - PageViews UInt8, - Duration UInt8, - Sign Int8 -) -ENGINE = CollapsingMergeTree(Sign) -ORDER BY UserID -``` - -Inserción de los datos: - -``` sql -INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) -``` - -``` sql -INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1),(4324182021466249494, 6, 185, 1) -``` - -Usamos dos `INSERT` consultas para crear dos partes de datos diferentes. Si insertamos los datos con una consulta, ClickHouse crea una parte de datos y nunca realizará ninguna fusión. - -Obtener los datos: - -``` sql -SELECT * FROM UAct -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -¿Qué vemos y dónde está colapsando? - -Con dos `INSERT` consultas, hemos creado 2 partes de datos. El `SELECT` la consulta se realizó en 2 hilos, y obtuvimos un orden aleatorio de filas. No se ha producido un colapso porque todavía no se había fusionado las partes de datos. ClickHouse fusiona parte de datos en un momento desconocido que no podemos predecir. - -Por lo tanto, necesitamos agregación: - -``` sql -SELECT - UserID, - sum(PageViews * Sign) AS PageViews, - sum(Duration * Sign) AS Duration -FROM UAct -GROUP BY UserID -HAVING sum(Sign) > 0 -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┐ -│ 4324182021466249494 │ 6 │ 185 │ -└─────────────────────┴───────────┴──────────┘ -``` - -Si no necesitamos agregación y queremos forzar el colapso, podemos usar `FINAL` modificador para `FROM` clausula. - -``` sql -SELECT * FROM UAct FINAL -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -Esta forma de seleccionar los datos es muy ineficiente. No lo use para mesas grandes. - -## Ejemplo de otro enfoque {#example-of-another-approach} - -Datos de ejemplo: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ -│ 4324182021466249494 │ -5 │ -146 │ -1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -La idea es que las fusiones tengan en cuenta solo los campos clave. Y en el “Cancel” línea podemos especificar valores negativos que igualan la versión anterior de la fila al sumar sin usar la columna Sign. Para este enfoque, es necesario cambiar el tipo de datos `PageViews`,`Duration` para almacenar valores negativos de UInt8 -\> Int16. - -``` sql -CREATE TABLE UAct -( - UserID UInt64, - PageViews Int16, - Duration Int16, - Sign Int8 -) -ENGINE = CollapsingMergeTree(Sign) -ORDER BY UserID -``` - -Vamos a probar el enfoque: - -``` sql -insert into UAct values(4324182021466249494, 5, 146, 1); -insert into UAct values(4324182021466249494, -5, -146, -1); -insert into UAct values(4324182021466249494, 6, 185, 1); - -select * from UAct final; // avoid using final in production (just for a test or small tables) -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -``` sql -SELECT - UserID, - sum(PageViews) AS PageViews, - sum(Duration) AS Duration -FROM UAct -GROUP BY UserID -```text -┌──────────────UserID─┬─PageViews─┬─Duration─┐ -│ 4324182021466249494 │ 6 │ 185 │ -└─────────────────────┴───────────┴──────────┘ -``` - -``` sqk -select count() FROM UAct -``` - -``` text -┌─count()─┐ -│ 3 │ -└─────────┘ -``` - -``` sql -optimize table UAct final; - -select * FROM UAct -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md deleted file mode 100644 index 6cbc0a9192e..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ /dev/null @@ -1,127 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 32 -toc_title: "Clave de partici\xF3n personalizada" ---- - -# Clave de partición personalizada {#custom-partitioning-key} - -La partición está disponible para el [Método de codificación de datos:](mergetree.md) mesas familiares (incluyendo [repetición](replication.md) tabla). [Vistas materializadas](../special/materializedview.md#materializedview) basado en tablas MergeTree soporte de particionamiento, también. - -Una partición es una combinación lógica de registros en una tabla por un criterio especificado. Puede establecer una partición por un criterio arbitrario, como por mes, por día o por tipo de evento. Cada partición se almacena por separado para simplificar las manipulaciones de estos datos. Al acceder a los datos, ClickHouse utiliza el subconjunto más pequeño de particiones posible. - -La partición se especifica en el `PARTITION BY expr` cláusula cuando [creando una tabla](mergetree.md#table_engine-mergetree-creating-a-table). La clave de partición puede ser cualquier expresión de las columnas de la tabla. Por ejemplo, para especificar la partición por mes, utilice la expresión `toYYYYMM(date_column)`: - -``` sql -CREATE TABLE visits -( - VisitDate Date, - Hour UInt8, - ClientID UUID -) -ENGINE = MergeTree() -PARTITION BY toYYYYMM(VisitDate) -ORDER BY Hour; -``` - -La clave de partición también puede ser una tupla de expresiones (similar a la [clave primaria](mergetree.md#primary-keys-and-indexes-in-queries)). Por ejemplo: - -``` sql -ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Sign) -PARTITION BY (toMonday(StartDate), EventType) -ORDER BY (CounterID, StartDate, intHash32(UserID)); -``` - -En este ejemplo, establecemos la partición por los tipos de eventos que se produjeron durante la semana actual. - -Al insertar datos nuevos en una tabla, estos datos se almacenan como una parte separada (porción) ordenada por la clave principal. En 10-15 minutos después de insertar, las partes de la misma partición se fusionan en toda la parte. - -!!! info "INFO" - Una combinación solo funciona para partes de datos que tienen el mismo valor para la expresión de partición. Esto significa **no deberías hacer particiones demasiado granulares** (más de un millar de particiones). De lo contrario, el `SELECT` consulta funciona mal debido a un número excesivamente grande de archivos en el sistema de archivos y descriptores de archivos abiertos. - -Utilice el [sistema.parte](../../../operations/system-tables.md#system_tables-parts) tabla para ver las partes y particiones de la tabla. Por ejemplo, supongamos que tenemos un `visits` tabla con partición por mes. Vamos a realizar el `SELECT` consulta para el `system.parts` tabla: - -``` sql -SELECT - partition, - name, - active -FROM system.parts -WHERE table = 'visits' -``` - -``` text -┌─partition─┬─name───────────┬─active─┐ -│ 201901 │ 201901_1_3_1 │ 0 │ -│ 201901 │ 201901_1_9_2 │ 1 │ -│ 201901 │ 201901_8_8_0 │ 0 │ -│ 201901 │ 201901_9_9_0 │ 0 │ -│ 201902 │ 201902_4_6_1 │ 1 │ -│ 201902 │ 201902_10_10_0 │ 1 │ -│ 201902 │ 201902_11_11_0 │ 1 │ -└───────────┴────────────────┴────────┘ -``` - -El `partition` columna contiene los nombres de las particiones. Hay dos particiones en este ejemplo: `201901` y `201902`. Puede utilizar este valor de columna para especificar el nombre de partición en [ALTER … PARTITION](#alter_manipulations-with-partitions) consulta. - -El `name` columna contiene los nombres de las partes de datos de partición. Puede utilizar esta columna para especificar el nombre de la pieza [ALTER ATTACH PART](#alter_attach-partition) consulta. - -Vamos a desglosar el nombre de la primera parte: `201901_1_3_1`: - -- `201901` es el nombre de la partición. -- `1` es el número mínimo del bloque de datos. -- `3` es el número máximo del bloque de datos. -- `1` es el nivel de fragmento (la profundidad del árbol de fusión del que se forma). - -!!! info "INFO" - Las partes de las tablas de tipo antiguo tienen el nombre: `20190117_20190123_2_2_0` (fecha mínima - fecha máxima - número de bloque mínimo - número de bloque máximo - nivel). - -El `active` columna muestra el estado de la pieza. `1` está activo; `0` está inactivo. Las partes inactivas son, por ejemplo, las partes de origen que quedan después de fusionarse con una parte más grande. Las partes de datos dañadas también se indican como inactivas. - -Como puede ver en el ejemplo, hay varias partes separadas de la misma partición (por ejemplo, `201901_1_3_1` y `201901_1_9_2`). Esto significa que estas partes aún no están fusionadas. ClickHouse combina las partes insertadas de datos periódicamente, aproximadamente 15 minutos después de la inserción. Además, puede realizar una fusión no programada utilizando el [OPTIMIZE](../../../sql-reference/statements/misc.md#misc_operations-optimize) consulta. Ejemplo: - -``` sql -OPTIMIZE TABLE visits PARTITION 201902; -``` - -``` text -┌─partition─┬─name───────────┬─active─┐ -│ 201901 │ 201901_1_3_1 │ 0 │ -│ 201901 │ 201901_1_9_2 │ 1 │ -│ 201901 │ 201901_8_8_0 │ 0 │ -│ 201901 │ 201901_9_9_0 │ 0 │ -│ 201902 │ 201902_4_6_1 │ 0 │ -│ 201902 │ 201902_4_11_2 │ 1 │ -│ 201902 │ 201902_10_10_0 │ 0 │ -│ 201902 │ 201902_11_11_0 │ 0 │ -└───────────┴────────────────┴────────┘ -``` - -Las partes inactivas se eliminarán aproximadamente 10 minutos después de la fusión. - -Otra forma de ver un conjunto de partes y particiones es ir al directorio de la tabla: `/var/lib/clickhouse/data///`. Por ejemplo: - -``` bash -/var/lib/clickhouse/data/default/visits$ ls -l -total 40 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 201901_1_3_1 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 16:17 201901_1_9_2 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 15:52 201901_8_8_0 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 15:52 201901_9_9_0 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 16:17 201902_10_10_0 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 16:17 201902_11_11_0 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 16:19 201902_4_11_2 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 5 12:09 201902_4_6_1 -drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached -``` - -Carpeta ‘201901_1_1_0’, ‘201901_1_7_1’ y así sucesivamente son los directorios de las partes. Cada parte se relaciona con una partición correspondiente y contiene datos solo para un mes determinado (la tabla de este ejemplo tiene particiones por mes). - -El `detached` el directorio contiene partes que se separaron de la tabla utilizando el [DETACH](../../../sql-reference/statements/alter.md#alter_detach-partition) consulta. Las partes dañadas también se mueven a este directorio, en lugar de eliminarse. El servidor no utiliza las piezas del `detached` directory. You can add, delete, or modify the data in this directory at any time – the server will not know about this until you run the [ATTACH](../../../sql-reference/statements/alter.md#alter_attach-partition) consulta. - -Tenga en cuenta que en el servidor operativo, no puede cambiar manualmente el conjunto de piezas o sus datos en el sistema de archivos, ya que el servidor no lo sabrá. Para tablas no replicadas, puede hacer esto cuando se detiene el servidor, pero no se recomienda. Para tablas replicadas, el conjunto de piezas no se puede cambiar en ningún caso. - -ClickHouse le permite realizar operaciones con las particiones: eliminarlas, copiar de una tabla a otra o crear una copia de seguridad. Consulte la lista de todas las operaciones en la sección [Manipulaciones con particiones y piezas](../../../sql-reference/statements/alter.md#alter_manipulations-with-partitions). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/custom_partitioning_key/) diff --git a/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md deleted file mode 100644 index d33ddcebac2..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/graphitemergetree.md +++ /dev/null @@ -1,174 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: GraphiteMergeTree ---- - -# GraphiteMergeTree {#graphitemergetree} - -Este motor está diseñado para el adelgazamiento y la agregación / promedio (rollup) [Grafito](http://graphite.readthedocs.io/en/latest/index.html) datos. Puede ser útil para los desarrolladores que desean usar ClickHouse como almacén de datos para Graphite. - -Puede usar cualquier motor de tabla ClickHouse para almacenar los datos de Graphite si no necesita un paquete acumulativo, pero si necesita un paquete acumulativo, use `GraphiteMergeTree`. El motor reduce el volumen de almacenamiento y aumenta la eficiencia de las consultas de Graphite. - -El motor hereda propiedades de [Método de codificación de datos:](mergetree.md). - -## Creación de una tabla {#creating-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - Path String, - Time DateTime, - Value , - Version - ... -) ENGINE = GraphiteMergeTree(config_section) -[PARTITION BY expr] -[ORDER BY expr] -[SAMPLE BY expr] -[SETTINGS name=value, ...] -``` - -Vea una descripción detallada del [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) consulta. - -Una tabla para los datos de grafito debe tener las siguientes columnas para los siguientes datos: - -- Nombre métrico (sensor de grafito). Tipo de datos: `String`. - -- Tiempo de medición de la métrica. Tipo de datos: `DateTime`. - -- Valor de la métrica. Tipo de datos: cualquier numérico. - -- Versión de la métrica. Tipo de datos: cualquier numérico. - - ClickHouse guarda las filas con la versión más alta o la última escrita si las versiones son las mismas. Otras filas se eliminan durante la fusión de partes de datos. - -Los nombres de estas columnas deben establecerse en la configuración acumulativa. - -**GraphiteMergeTree parámetros** - -- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. - -**Cláusulas de consulta** - -Al crear un `GraphiteMergeTree` mesa, la misma [clausula](mergetree.md#table_engine-mergetree-creating-a-table) se requieren, como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No use este método en proyectos nuevos y, si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - EventDate Date, - Path String, - Time DateTime, - Value , - Version - ... -) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section) -``` - -Todos los parámetros excepto `config_section` el mismo significado que en `MergeTree`. - -- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. - -
- -## Configuración acumulativa {#rollup-configuration} - -La configuración del paquete acumulativo está definida por [graphite_rollup](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-graphite) parámetro en la configuración del servidor. El nombre del parámetro podría ser cualquiera. Puede crear varias configuraciones y usarlas para diferentes tablas. - -Estructura de configuración Rollup: - - required-columns - patterns - -### Columnas requeridas {#required-columns} - -- `path_column_name` — The name of the column storing the metric name (Graphite sensor). Default value: `Path`. -- `time_column_name` — The name of the column storing the time of measuring the metric. Default value: `Time`. -- `value_column_name` — The name of the column storing the value of the metric at the time set in `time_column_name`. Valor predeterminado: `Value`. -- `version_column_name` — The name of the column storing the version of the metric. Default value: `Timestamp`. - -### Patrón {#patterns} - -Estructura del `patterns` apartado: - -``` text -pattern - regexp - function -pattern - regexp - age + precision - ... -pattern - regexp - function - age + precision - ... -pattern - ... -default - function - age + precision - ... -``` - -!!! warning "Atención" - Los patrones deben ser estrictamente ordenados: - - 1. Patterns without `function` or `retention`. - 1. Patterns with both `function` and `retention`. - 1. Pattern `default`. - -Al procesar una fila, ClickHouse comprueba las reglas en el `pattern` apartado. Cada uno de `pattern` (incluir `default` secciones pueden contener `function` parámetro para la agregación, `retention` parámetros o ambos. Si el nombre de la métrica coincide con `regexp`, las reglas de la `pattern` sección (o secciones); de lo contrario, las reglas de la `default` sección se utilizan. - -Campos para `pattern` y `default` apartado: - -- `regexp`– A pattern for the metric name. -- `age` – The minimum age of the data in seconds. -- `precision`– How precisely to define the age of the data in seconds. Should be a divisor for 86400 (seconds in a day). -- `function` – The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. - -### Ejemplo de configuración {#configuration-example} - -``` xml - - Version - - click_cost - any - - 0 - 5 - - - 86400 - 60 - - - - max - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/graphitemergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/index.md b/docs/es/engines/table-engines/mergetree-family/index.md deleted file mode 100644 index 359d58b2ff1..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Familia MergeTree -toc_priority: 28 ---- - - diff --git a/docs/es/engines/table-engines/mergetree-family/mergetree.md b/docs/es/engines/table-engines/mergetree-family/mergetree.md deleted file mode 100644 index a4bab840b52..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/mergetree.md +++ /dev/null @@ -1,654 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 30 -toc_title: "M\xE9todo de codificaci\xF3n de datos:" ---- - -# Método de codificación de datos: {#table_engines-mergetree} - -El `MergeTree` motor y otros motores de esta familia (`*MergeTree`) son los motores de mesa ClickHouse más robustos. - -Motores en el `MergeTree` familia están diseñados para insertar una gran cantidad de datos en una tabla. Los datos se escriben rápidamente en la tabla parte por parte, luego se aplican reglas para fusionar las partes en segundo plano. Este método es mucho más eficiente que reescribir continuamente los datos en almacenamiento durante la inserción. - -Principales características: - -- Almacena datos ordenados por clave principal. - - Esto le permite crear un pequeño índice disperso que ayuda a encontrar datos más rápido. - -- Las particiones se pueden utilizar si [clave de partición](custom-partitioning-key.md) se especifica. - - ClickHouse admite ciertas operaciones con particiones que son más efectivas que las operaciones generales en los mismos datos con el mismo resultado. ClickHouse también corta automáticamente los datos de partición donde se especifica la clave de partición en la consulta. Esto también mejora el rendimiento de las consultas. - -- Soporte de replicación de datos. - - La familia de `ReplicatedMergeTree` proporciona la replicación de datos. Para obtener más información, consulte [Replicación de datos](replication.md). - -- Soporte de muestreo de datos. - - Si es necesario, puede establecer el método de muestreo de datos en la tabla. - -!!! info "INFO" - El [Fusionar](../special/merge.md#merge) el motor no pertenece al `*MergeTree` familia. - -## Creación de una tabla {#table_engine-mergetree-creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], - ... - INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, - INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 -) ENGINE = MergeTree() -[PARTITION BY expr] -[ORDER BY expr] -[PRIMARY KEY expr] -[SAMPLE BY expr] -[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros, consulte [Descripción de la consulta CREATE](../../../sql-reference/statements/create.md). - -!!! note "Nota" - `INDEX` es una característica experimental, ver [Índices de saltos de datos](#table_engine-mergetree-data_skipping-indexes). - -### Cláusulas de consulta {#mergetree-query-clauses} - -- `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. El `MergeTree` el motor no tiene parámetros. - -- `PARTITION BY` — The [clave de partición](custom-partitioning-key.md). - - Para particionar por mes, utilice el `toYYYYMM(date_column)` expresión, donde `date_column` es una columna con una fecha del tipo [Fecha](../../../sql-reference/data-types/date.md). Los nombres de partición aquí tienen el `"YYYYMM"` formato. - -- `ORDER BY` — The sorting key. - - Una tupla de columnas o expresiones arbitrarias. Ejemplo: `ORDER BY (CounterID, EventDate)`. - -- `PRIMARY KEY` — The primary key if it [difiere de la clave de clasificación](#choosing-a-primary-key-that-differs-from-the-sorting-key). - - De forma predeterminada, la clave principal es la misma que la clave de ordenación (que se especifica `ORDER BY` clausula). Por lo tanto, en la mayoría de los casos no es necesario especificar un `PRIMARY KEY` clausula. - -- `SAMPLE BY` — An expression for sampling. - - Si se utiliza una expresión de muestreo, la clave principal debe contenerla. Ejemplo: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. - -- `TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [entre discos y volúmenes](#table_engine-mergetree-multiple-volumes). - - La expresión debe tener una `Date` o `DateTime` columna como resultado. Ejemplo: - `TTL date + INTERVAL 1 DAY` - - Tipo de regla `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'` especifica una acción que debe realizarse con la pieza si la expresión está satisfecha (alcanza la hora actual): eliminación de filas caducadas, mover una pieza (si la expresión está satisfecha para todas las filas de una pieza) al disco especificado (`TO DISK 'xxx'`) o al volumen (`TO VOLUME 'xxx'`). El tipo predeterminado de la regla es la eliminación (`DELETE`). Se puede especificar una lista de varias reglas, pero no debe haber más de una `DELETE` regla. - - Para obtener más información, consulte [TTL para columnas y tablas](#table_engine-mergetree-ttl) - -- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`: - - - `index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Almacenamiento de datos](#mergetree-data-storage). - - `index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Almacenamiento de datos](#mergetree-data-storage). - - `enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` configuración. Antes de la versión 19.11, sólo existía el `index_granularity` ajuste para restringir el tamaño del gránulo. El `index_granularity_bytes` mejora el rendimiento de ClickHouse al seleccionar datos de tablas con filas grandes (decenas y cientos de megabytes). Si tiene tablas con filas grandes, puede habilitar esta configuración para que las tablas mejoren la eficiencia de `SELECT` consulta. - - `use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, entonces ZooKeeper almacena menos datos. Para obtener más información, consulte [descripción del ajuste](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) en “Server configuration parameters”. - - `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse lee y escribe los datos en el disco de almacenamiento utilizando la interfaz de E / S directa (`O_DIRECT` opcion). Si `min_merge_bytes_to_use_direct_io = 0`, entonces la E/S directa está deshabilitada. Valor predeterminado: `10 * 1024 * 1024 * 1024` byte. - - - `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day). - - `write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don't turn it off. - - `merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - - `storage_policy` — Storage policy. See [Uso de varios dispositivos de bloque para el almacenamiento de datos](#table_engine-mergetree-multiple-volumes). - -**Ejemplo de configuración de secciones** - -``` sql -ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 -``` - -En el ejemplo, configuramos la partición por mes. - -También establecemos una expresión para el muestreo como un hash por el ID de usuario. Esto le permite pseudoaleatorizar los datos en la tabla para cada `CounterID` y `EventDate`. Si define un [SAMPLE](../../../sql-reference/statements/select/sample.md#select-sample-clause) cláusula al seleccionar los datos, ClickHouse devolverá una muestra de datos pseudoaleatoria uniforme para un subconjunto de usuarios. - -El `index_granularity` se puede omitir porque 8192 es el valor predeterminado. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No utilice este método en nuevos proyectos. Si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) -``` - -**Parámetros MergeTree()** - -- `date-column` — The name of a column of the [Fecha](../../../sql-reference/data-types/date.md) tipo. ClickHouse crea automáticamente particiones por mes en función de esta columna. Los nombres de partición están en el `"YYYYMM"` formato. -- `sampling_expression` — An expression for sampling. -- `(primary, key)` — Primary key. Type: [Tupla()](../../../sql-reference/data-types/tuple.md) -- `index_granularity` — The granularity of an index. The number of data rows between the “marks” de un índice. El valor 8192 es apropiado para la mayoría de las tareas. - -**Ejemplo** - -``` sql -MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)), 8192) -``` - -El `MergeTree` engine se configura de la misma manera que en el ejemplo anterior para el método de configuración del motor principal. -
- -## Almacenamiento de datos {#mergetree-data-storage} - -Una tabla consta de partes de datos ordenadas por clave principal. - -Cuando se insertan datos en una tabla, se crean partes de datos separadas y cada una de ellas se ordena lexicográficamente por clave principal. Por ejemplo, si la clave principal es `(CounterID, Date)`, los datos en la parte se ordenan por `CounterID`, y dentro de cada `CounterID` es ordenado por `Date`. - -Los datos que pertenecen a diferentes particiones se separan en diferentes partes. En el fondo, ClickHouse combina partes de datos para un almacenamiento más eficiente. Las piezas que pertenecen a particiones diferentes no se fusionan. El mecanismo de combinación no garantiza que todas las filas con la misma clave principal estén en la misma parte de datos. - -Cada parte de datos se divide lógicamente en gránulos. Un gránulo es el conjunto de datos indivisibles más pequeño que ClickHouse lee al seleccionar datos. ClickHouse no divide filas o valores, por lo que cada gránulo siempre contiene un número entero de filas. La primera fila de un gránulo está marcada con el valor de la clave principal de la fila. Para cada parte de datos, ClickHouse crea un archivo de índice que almacena las marcas. Para cada columna, ya sea en la clave principal o no, ClickHouse también almacena las mismas marcas. Estas marcas le permiten encontrar datos directamente en archivos de columnas. - -El tamaño del gránulo es restringido por `index_granularity` y `index_granularity_bytes` configuración del motor de tabla. El número de filas en un gránulo se encuentra en el `[1, index_granularity]` rango, dependiendo del tamaño de las filas. El tamaño de un gránulo puede exceder `index_granularity_bytes` si el tamaño de una sola fila es mayor que el valor de la configuración. En este caso, el tamaño del gránulo es igual al tamaño de la fila. - -## Claves e índices principales en consultas {#primary-keys-and-indexes-in-queries} - -Tome el `(CounterID, Date)` clave primaria como ejemplo. En este caso, la clasificación y el índice se pueden ilustrar de la siguiente manera: - - Whole data: [---------------------------------------------] - CounterID: [aaaaaaaaaaaaaaaaaabbbbcdeeeeeeeeeeeeefgggggggghhhhhhhhhiiiiiiiiikllllllll] - Date: [1111111222222233331233211111222222333211111112122222223111112223311122333] - Marks: | | | | | | | | | | | - a,1 a,2 a,3 b,3 e,2 e,3 g,1 h,2 i,1 i,3 l,3 - Marks numbers: 0 1 2 3 4 5 6 7 8 9 10 - -Si la consulta de datos especifica: - -- `CounterID in ('a', 'h')`, el servidor lee los datos en los rangos de marcas `[0, 3)` y `[6, 8)`. -- `CounterID IN ('a', 'h') AND Date = 3`, el servidor lee los datos en los rangos de marcas `[1, 3)` y `[7, 8)`. -- `Date = 3`, el servidor lee los datos en el rango de marcas `[1, 10]`. - -Los ejemplos anteriores muestran que siempre es más efectivo usar un índice que un análisis completo. - -Un índice disperso permite leer datos adicionales. Al leer un único rango de la clave primaria, hasta `index_granularity * 2` se pueden leer filas adicionales en cada bloque de datos. - -Los índices dispersos le permiten trabajar con una gran cantidad de filas de tabla, porque en la mayoría de los casos, dichos índices caben en la RAM de la computadora. - -ClickHouse no requiere una clave principal única. Puede insertar varias filas con la misma clave principal. - -### Selección de la clave principal {#selecting-the-primary-key} - -El número de columnas en la clave principal no está explícitamente limitado. Dependiendo de la estructura de datos, puede incluir más o menos columnas en la clave principal. Esto puede: - -- Mejorar el rendimiento de un índice. - - Si la clave principal es `(a, b)`, a continuación, añadir otra columna `c` mejorará el rendimiento si se cumplen las siguientes condiciones: - - - Hay consultas con una condición en la columna `c`. - - Rangos de datos largos (varias veces más `index_granularity`) con valores idénticos para `(a, b)` son comunes. En otras palabras, al agregar otra columna le permite omitir rangos de datos bastante largos. - -- Mejorar la compresión de datos. - - ClickHouse ordena los datos por clave principal, por lo que cuanto mayor sea la consistencia, mejor será la compresión. - -- Proporcione una lógica adicional al fusionar partes de datos en el [ColapsarMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) y [SummingMergeTree](summingmergetree.md) motor. - - En este caso tiene sentido especificar el *clave de clasificación* que es diferente de la clave principal. - -Una clave principal larga afectará negativamente al rendimiento de la inserción y al consumo de memoria, pero las columnas adicionales de la clave principal no afectarán al rendimiento de ClickHouse durante `SELECT` consulta. - -### Elegir una clave principal que difiere de la clave de ordenación {#choosing-a-primary-key-that-differs-from-the-sorting-key} - -Es posible especificar una clave principal (una expresión con valores que se escriben en el archivo de índice para cada marca) que es diferente de la clave de ordenación (una expresión para ordenar las filas en partes de datos). En este caso, la tupla de expresión de clave primaria debe ser un prefijo de la tupla de expresión de clave de ordenación. - -Esta característica es útil cuando se [SummingMergeTree](summingmergetree.md) y -[AgregaciónMergeTree](aggregatingmergetree.md) motores de mesa. En un caso común cuando se utilizan estos motores, la tabla tiene dos tipos de columnas: *cota* y *medida*. Las consultas típicas agregan valores de columnas de medida con `GROUP BY` y filtrado por dimensiones. Debido a que SummingMergeTree y AggregatingMergeTree agregan filas con el mismo valor de la clave de ordenación, es natural agregarle todas las dimensiones. Como resultado, la expresión de clave consta de una larga lista de columnas y esta lista debe actualizarse con frecuencia con las dimensiones recién agregadas. - -En este caso, tiene sentido dejar solo unas pocas columnas en la clave principal que proporcionarán análisis de rango eficientes y agregarán las columnas de dimensión restantes a la tupla de clave de clasificación. - -[ALTER](../../../sql-reference/statements/alter.md) de la clave de ordenación es una operación ligera porque cuando se agrega una nueva columna simultáneamente a la tabla y a la clave de ordenación, las partes de datos existentes no necesitan ser cambiadas. Dado que la clave de ordenación anterior es un prefijo de la nueva clave de ordenación y no hay datos en la columna recién agregada, los datos se ordenan tanto por las claves de ordenación antiguas como por las nuevas en el momento de la modificación de la tabla. - -### Uso de índices y particiones en consultas {#use-of-indexes-and-partitions-in-queries} - -Para `SELECT` consultas, ClickHouse analiza si se puede usar un índice. Se puede usar un índice si el `WHERE/PREWHERE` clause tiene una expresión (como uno de los elementos de conjunción, o enteramente) que representa una operación de comparación de igualdad o desigualdad, o si tiene `IN` o `LIKE` con un prefijo fijo en columnas o expresiones que están en la clave principal o clave de partición, o en ciertas funciones parcialmente repetitivas de estas columnas, o relaciones lógicas de estas expresiones. - -Por lo tanto, es posible ejecutar rápidamente consultas en uno o varios rangos de la clave principal. En este ejemplo, las consultas serán rápidas cuando se ejecuten para una etiqueta de seguimiento específica, para una etiqueta y un intervalo de fechas específicos, para una etiqueta y una fecha específicas, para varias etiquetas con un intervalo de fechas, etc. - -Veamos el motor configurado de la siguiente manera: - - ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTINGS index_granularity=8192 - -En este caso, en consultas: - -``` sql -SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34 -SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42) -SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01')) -``` - -ClickHouse utilizará el índice de clave principal para recortar datos incorrectos y la clave de partición mensual para recortar particiones que están en intervalos de fechas incorrectos. - -Las consultas anteriores muestran que el índice se usa incluso para expresiones complejas. La lectura de la tabla está organizada de modo que el uso del índice no puede ser más lento que un escaneo completo. - -En el siguiente ejemplo, el índice no se puede usar. - -``` sql -SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' -``` - -Para comprobar si ClickHouse puede usar el índice al ejecutar una consulta, use la configuración [Fecha de nacimiento](../../../operations/settings/settings.md#settings-force_index_by_date) y [force_primary_key](../../../operations/settings/settings.md). - -La clave para particionar por mes permite leer solo aquellos bloques de datos que contienen fechas del rango adecuado. En este caso, el bloque de datos puede contener datos para muchas fechas (hasta un mes). Dentro de un bloque, los datos se ordenan por clave principal, que puede no contener la fecha como la primera columna. Debido a esto, el uso de una consulta con solo una condición de fecha que no especifica el prefijo de clave principal hará que se lean más datos que para una sola fecha. - -### Uso del índice para claves primarias parcialmente monótonas {#use-of-index-for-partially-monotonic-primary-keys} - -Considere, por ejemplo, los días del mes. Ellos forman un [monótona secuencia](https://en.wikipedia.org/wiki/Monotonic_function) durante un mes, pero no monótono durante períodos más prolongados. Esta es una secuencia parcialmente monotónica. Si un usuario crea la tabla con clave primaria parcialmente monótona, ClickHouse crea un índice disperso como de costumbre. Cuando un usuario selecciona datos de este tipo de tabla, ClickHouse analiza las condiciones de consulta. Si el usuario desea obtener datos entre dos marcas del índice y ambas marcas caen dentro de un mes, ClickHouse puede usar el índice en este caso particular porque puede calcular la distancia entre los parámetros de una consulta y las marcas de índice. - -ClickHouse no puede usar un índice si los valores de la clave principal en el rango de parámetros de consulta no representan una secuencia monotónica. En este caso, ClickHouse utiliza el método de análisis completo. - -ClickHouse usa esta lógica no solo para secuencias de días del mes, sino para cualquier clave principal que represente una secuencia parcialmente monotónica. - -### Índices de saltos de datos (experimental) {#table_engine-mergetree-data_skipping-indexes} - -La declaración de índice se encuentra en la sección de columnas del `CREATE` consulta. - -``` sql -INDEX index_name expr TYPE type(...) GRANULARITY granularity_value -``` - -Para tablas de la `*MergeTree` familia, se pueden especificar índices de omisión de datos. - -Estos índices agregan cierta información sobre la expresión especificada en bloques, que consisten en `granularity_value` gránulos (el tamaño del gránulo se especifica utilizando el `index_granularity` ajuste en el motor de la tabla). Entonces estos agregados se usan en `SELECT` consultas para reducir la cantidad de datos a leer desde el disco omitiendo grandes bloques de datos donde el `where` consulta no puede ser satisfecha. - -**Ejemplo** - -``` sql -CREATE TABLE table_name -( - u64 UInt64, - i32 Int32, - s String, - ... - INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3, - INDEX b (u64 * length(s)) TYPE set(1000) GRANULARITY 4 -) ENGINE = MergeTree() -... -``` - -ClickHouse puede utilizar los índices del ejemplo para reducir la cantidad de datos que se leen desde el disco en las siguientes consultas: - -``` sql -SELECT count() FROM table WHERE s < 'z' -SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 -``` - -#### Tipos de índices disponibles {#available-types-of-indices} - -- `minmax` - - Almacena los extremos de la expresión especificada (si la expresión `tuple`, entonces almacena extremos para cada elemento de `tuple`), utiliza información almacenada para omitir bloques de datos como la clave principal. - -- `set(max_rows)` - - Almacena valores únicos de la expresión especificada (no más de `max_rows` filas, `max_rows=0` medio “no limits”). Utiliza los valores para comprobar si `WHERE` expresión no es satisfactorio en un bloque de datos. - -- `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` - - Tiendas a [Filtro de floración](https://en.wikipedia.org/wiki/Bloom_filter) que contiene todos los ngrams de un bloque de datos. Funciona solo con cadenas. Puede ser utilizado para la optimización de `equals`, `like` y `in` expresiones. - - - `n` — ngram size, - - `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well). - - `number_of_hash_functions` — The number of hash functions used in the Bloom filter. - - `random_seed` — The seed for Bloom filter hash functions. - -- `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` - - Lo mismo que `ngrambf_v1`, pero almacena tokens en lugar de ngrams. Los tokens son secuencias separadas por caracteres no alfanuméricos. - -- `bloom_filter([false_positive])` — Stores a [Filtro de floración](https://en.wikipedia.org/wiki/Bloom_filter) para las columnas especificadas. - - Opcional `false_positive` parámetro es la probabilidad de recibir una respuesta falsa positiva del filtro. Valores posibles: (0, 1). Valor predeterminado: 0.025. - - Tipos de datos admitidos: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`. - - Las siguientes funciones pueden usarlo: [igual](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [en](../../../sql-reference/functions/in-functions.md), [noEn](../../../sql-reference/functions/in-functions.md), [tener](../../../sql-reference/functions/array-functions.md). - - - -``` sql -INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4 -INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4 -INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4 -``` - -#### Funciones de apoyo {#functions-support} - -Condiciones en el `WHERE` cláusula contiene llamadas de las funciones que operan con columnas. Si la columna forma parte de un índice, ClickHouse intenta usar este índice al realizar las funciones. ClickHouse admite diferentes subconjuntos de funciones para usar índices. - -El `set` index se puede utilizar con todas las funciones. Subconjuntos de funciones para otros índices se muestran en la siguiente tabla. - -| Función (operador) / Índice | clave primaria | minmax | Descripción | Sistema abierto. | bloom_filter | -|----------------------------------------------------------------------------------------------------------|----------------|--------|-------------|------------------|---------------| -| [igual (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [como](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗ | -| [No como](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✗ | ✗ | -| [Comienza con](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [Finaliza con](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | -| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [en](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [noEn](../../../sql-reference/functions/in-functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [menos (\<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [mayor (\>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [menosOrEquals (\<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [mayorOrEquals (\>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [vaciar](../../../sql-reference/functions/array-functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](../../../sql-reference/functions/array-functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | - -Las funciones con un argumento constante que es menor que el tamaño de ngram no pueden ser utilizadas por `ngrambf_v1` para la optimización de consultas. - -Los filtros Bloom pueden tener coincidencias falsas positivas, por lo que `ngrambf_v1`, `tokenbf_v1`, y `bloom_filter` los índices no se pueden usar para optimizar consultas donde se espera que el resultado de una función sea falso, por ejemplo: - -- Puede ser optimizado: - - `s LIKE '%test%'` - - `NOT s NOT LIKE '%test%'` - - `s = 1` - - `NOT s != 1` - - `startsWith(s, 'test')` -- No se puede optimizar: - - `NOT s LIKE '%test%'` - - `s NOT LIKE '%test%'` - - `NOT s = 1` - - `s != 1` - - `NOT startsWith(s, 'test')` - -## Acceso a datos simultáneos {#concurrent-data-access} - -Para el acceso simultáneo a tablas, usamos versiones múltiples. En otras palabras, cuando una tabla se lee y actualiza simultáneamente, los datos se leen de un conjunto de partes que está actualizado en el momento de la consulta. No hay cerraduras largas. Las inserciones no se interponen en el camino de las operaciones de lectura. - -La lectura de una tabla se paralela automáticamente. - -## TTL para columnas y tablas {#table_engine-mergetree-ttl} - -Determina la duración de los valores. - -El `TTL` se puede establecer para toda la tabla y para cada columna individual. TTL de nivel de tabla también puede especificar la lógica de movimiento automático de datos entre discos y volúmenes. - -Las expresiones deben evaluar [Fecha](../../../sql-reference/data-types/date.md) o [FechaHora](../../../sql-reference/data-types/datetime.md) tipo de datos. - -Ejemplo: - -``` sql -TTL time_column -TTL time_column + interval -``` - -Definir `interval`, utilizar [intervalo de tiempo](../../../sql-reference/operators/index.md#operators-datetime) operador. - -``` sql -TTL date_time + INTERVAL 1 MONTH -TTL date_time + INTERVAL 15 HOUR -``` - -### Columna TTL {#mergetree-column-ttl} - -Cuando los valores de la columna caducan, ClickHouse los reemplaza con los valores predeterminados para el tipo de datos de columna. Si todos los valores de columna en la parte de datos caducan, ClickHouse elimina esta columna de la parte de datos en un sistema de archivos. - -El `TTL` cláusula no se puede utilizar para columnas clave. - -Ejemplos: - -Creación de una tabla con TTL - -``` sql -CREATE TABLE example_table -( - d DateTime, - a Int TTL d + INTERVAL 1 MONTH, - b Int TTL d + INTERVAL 1 MONTH, - c String -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(d) -ORDER BY d; -``` - -Adición de TTL a una columna de una tabla existente - -``` sql -ALTER TABLE example_table - MODIFY COLUMN - c String TTL d + INTERVAL 1 DAY; -``` - -Modificación de TTL de la columna - -``` sql -ALTER TABLE example_table - MODIFY COLUMN - c String TTL d + INTERVAL 1 MONTH; -``` - -### Tabla TTL {#mergetree-table-ttl} - -La tabla puede tener una expresión para la eliminación de filas caducadas y varias expresiones para el movimiento automático de partes entre [discos o volúmenes](#table_engine-mergetree-multiple-volumes). Cuando las filas de la tabla caducan, ClickHouse elimina todas las filas correspondientes. Para la entidad de movimiento de piezas, todas las filas de una pieza deben cumplir los criterios de expresión de movimiento. - -``` sql -TTL expr [DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'], ... -``` - -El tipo de regla TTL puede seguir cada expresión TTL. Afecta a una acción que debe realizarse una vez que se satisface la expresión (alcanza la hora actual): - -- `DELETE` - Eliminar filas caducadas (acción predeterminada); -- `TO DISK 'aaa'` - mover parte al disco `aaa`; -- `TO VOLUME 'bbb'` - mover parte al disco `bbb`. - -Ejemplos: - -Creación de una tabla con TTL - -``` sql -CREATE TABLE example_table -( - d DateTime, - a Int -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(d) -ORDER BY d -TTL d + INTERVAL 1 MONTH [DELETE], - d + INTERVAL 1 WEEK TO VOLUME 'aaa', - d + INTERVAL 2 WEEK TO DISK 'bbb'; -``` - -Modificación de TTL de la tabla - -``` sql -ALTER TABLE example_table - MODIFY TTL d + INTERVAL 1 DAY; -``` - -**Eliminación de datos** - -Los datos con un TTL caducado se eliminan cuando ClickHouse fusiona partes de datos. - -Cuando ClickHouse ve que los datos han caducado, realiza una combinación fuera de programación. Para controlar la frecuencia de tales fusiones, puede establecer `merge_with_ttl_timeout`. Si el valor es demasiado bajo, realizará muchas fusiones fuera de horario que pueden consumir muchos recursos. - -Si realiza el `SELECT` consulta entre fusiones, puede obtener datos caducados. Para evitarlo, use el [OPTIMIZE](../../../sql-reference/statements/misc.md#misc_operations-optimize) consulta antes `SELECT`. - -## Uso de varios dispositivos de bloque para el almacenamiento de datos {#table_engine-mergetree-multiple-volumes} - -### Implantación {#introduction} - -`MergeTree` Los motores de tablas familiares pueden almacenar datos en múltiples dispositivos de bloque. Por ejemplo, puede ser útil cuando los datos de una determinada tabla se dividen implícitamente en “hot” y “cold”. Los datos más recientes se solicitan regularmente, pero solo requieren una pequeña cantidad de espacio. Por el contrario, los datos históricos de cola gorda se solicitan raramente. Si hay varios discos disponibles, el “hot” los datos pueden estar ubicados en discos rápidos (por ejemplo, SSD NVMe o en memoria), mientras que “cold” datos - en los relativamente lentos (por ejemplo, HDD). - -La parte de datos es la unidad móvil mínima para `MergeTree`-mesas de motor. Los datos que pertenecen a una parte se almacenan en un disco. Las partes de datos se pueden mover entre discos en segundo plano (según la configuración del usuario) así como por medio de la [ALTER](../../../sql-reference/statements/alter.md#alter_move-partition) consulta. - -### Plazo {#terms} - -- Disk — Block device mounted to the filesystem. -- Default disk — Disk that stores the path specified in the [camino](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) configuración del servidor. -- Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). -- Storage policy — Set of volumes and the rules for moving data between them. - -Los nombres dados a las entidades descritas se pueden encontrar en las tablas del sistema, [sistema.almacenamiento_policies](../../../operations/system-tables.md#system_tables-storage_policies) y [sistema.disco](../../../operations/system-tables.md#system_tables-disks). Para aplicar una de las directivas de almacenamiento configuradas para una tabla, `storage_policy` establecimiento de `MergeTree`-mesas de la familia del motor. - -### Configuración {#table_engine-mergetree-multiple-volumes_configure} - -Los discos, los volúmenes y las políticas de almacenamiento deben declararse `` etiqueta ya sea en el archivo principal `config.xml` o en un archivo distinto en el `config.d` directorio. - -Estructura de configuración: - -``` xml - - - - /mnt/fast_ssd/clickhouse/ - - - /mnt/hdd1/clickhouse/ - 10485760 - - - /mnt/hdd2/clickhouse/ - 10485760 - - - ... - - - ... - -``` - -Tags: - -- `` — Disk name. Names must be different for all disks. -- `path` — path under which a server will store data (`data` y `shadow` carpetas), debe terminarse con ‘/’. -- `keep_free_space_bytes` — the amount of free disk space to be reserved. - -El orden de la definición del disco no es importante. - -Marcado de configuración de directivas de almacenamiento: - -``` xml - - ... - - - - - disk_name_from_disks_configuration - 1073741824 - - - - - - - 0.2 - - - - - - - - ... - -``` - -Tags: - -- `policy_name_N` — Policy name. Policy names must be unique. -- `volume_name_N` — Volume name. Volume names must be unique. -- `disk` — a disk within a volume. -- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volume's disks. -- `move_factor` — when the amount of available space gets lower than this factor, data automatically start to move on the next volume if any (by default, 0.1). - -Cofiguration ejemplos: - -``` xml - - ... - - - - - disk1 - disk2 - - - - - - - - fast_ssd - 1073741824 - - - disk1 - - - 0.2 - - - ... - -``` - -En un ejemplo dado, el `hdd_in_order` la política implementa el [Ronda-robin](https://en.wikipedia.org/wiki/Round-robin_scheduling) enfoque. Por lo tanto, esta política define solo un volumen (`single`), las partes de datos se almacenan en todos sus discos en orden circular. Dicha política puede ser bastante útil si hay varios discos similares montados en el sistema, pero RAID no está configurado. Tenga en cuenta que cada unidad de disco individual no es confiable y es posible que desee compensarlo con un factor de replicación de 3 o más. - -Si hay diferentes tipos de discos disponibles en el sistema, `moving_from_ssd_to_hdd` política se puede utilizar en su lugar. Volumen `hot` consta de un disco SSD (`fast_ssd`), y el tamaño máximo de una pieza que se puede almacenar en este volumen es de 1 GB. Todas las piezas con el tamaño más grande que 1GB serán almacenadas directamente en `cold` volumen, que contiene un disco duro `disk1`. -Además, una vez que el disco `fast_ssd` se llena en más del 80%, los datos se transferirán al `disk1` por un proceso en segundo plano. - -El orden de enumeración de volúmenes dentro de una directiva de almacenamiento es importante. Una vez que un volumen está sobrellenado, los datos se mueven al siguiente. El orden de la enumeración del disco también es importante porque los datos se almacenan en ellos por turnos. - -Al crear una tabla, se puede aplicarle una de las directivas de almacenamiento configuradas: - -``` sql -CREATE TABLE table_with_non_default_policy ( - EventDate Date, - OrderID UInt64, - BannerID UInt64, - SearchPhrase String -) ENGINE = MergeTree -ORDER BY (OrderID, BannerID) -PARTITION BY toYYYYMM(EventDate) -SETTINGS storage_policy = 'moving_from_ssd_to_hdd' -``` - -El `default` política de almacenamiento implica el uso de un solo volumen, que consiste en un solo disco dado en ``. Una vez que se crea una tabla, no se puede cambiar su política de almacenamiento. - -### Detalles {#details} - -En el caso de `MergeTree` tablas, los datos están llegando al disco de diferentes maneras: - -- Como resultado de un inserto (`INSERT` consulta). -- Durante las fusiones de fondo y [mutación](../../../sql-reference/statements/alter.md#alter-mutations). -- Al descargar desde otra réplica. -- Como resultado de la congelación de particiones [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition). - -En todos estos casos, excepto las mutaciones y la congelación de particiones, una pieza se almacena en un volumen y un disco de acuerdo con la política de almacenamiento dada: - -1. El primer volumen (en el orden de definición) que tiene suficiente espacio en disco para almacenar una pieza (`unreserved_space > current_part_size`) y permite almacenar partes de un tamaño determinado (`max_data_part_size_bytes > current_part_size`) se elige. -2. Dentro de este volumen, se elige ese disco que sigue al que se utilizó para almacenar el fragmento de datos anterior y que tiene espacio libre más que el tamaño de la pieza (`unreserved_space - keep_free_space_bytes > current_part_size`). - -Bajo el capó, las mutaciones y la congelación de particiones hacen uso de [enlaces duros](https://en.wikipedia.org/wiki/Hard_link). Los enlaces duros entre diferentes discos no son compatibles, por lo tanto, en tales casos las partes resultantes se almacenan en los mismos discos que los iniciales. - -En el fondo, las partes se mueven entre volúmenes en función de la cantidad de espacio libre (`move_factor` parámetro) según el orden en que se declaran los volúmenes en el archivo de configuración. -Los datos nunca se transfieren desde el último y al primero. Uno puede usar tablas del sistema [sistema.part_log](../../../operations/system-tables.md#system_tables-part-log) (campo `type = MOVE_PART`) y [sistema.parte](../../../operations/system-tables.md#system_tables-parts) (campo `path` y `disk`) para monitorear movimientos de fondo. Además, la información detallada se puede encontrar en los registros del servidor. - -El usuario puede forzar el movimiento de una pieza o una partición de un volumen a otro mediante la consulta [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition), todas las restricciones para las operaciones en segundo plano se tienen en cuenta. La consulta inicia un movimiento por sí misma y no espera a que se completen las operaciones en segundo plano. El usuario recibirá un mensaje de error si no hay suficiente espacio libre disponible o si no se cumple alguna de las condiciones requeridas. - -Mover datos no interfiere con la replicación de datos. Por lo tanto, se pueden especificar diferentes directivas de almacenamiento para la misma tabla en diferentes réplicas. - -Después de la finalización de las fusiones y mutaciones de fondo, las partes viejas se eliminan solo después de un cierto período de tiempo (`old_parts_lifetime`). -Durante este tiempo, no se mueven a otros volúmenes o discos. Por lo tanto, hasta que las partes finalmente se eliminen, aún se tienen en cuenta para la evaluación del espacio en disco ocupado. - -[Artículo Original](https://clickhouse.tech/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md deleted file mode 100644 index a1e95c5b5f4..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/replacingmergetree.md +++ /dev/null @@ -1,69 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 33 -toc_title: ReplacingMergeTree ---- - -# ReplacingMergeTree {#replacingmergetree} - -El motor difiere de [Método de codificación de datos:](mergetree.md#table_engines-mergetree) en que elimina las entradas duplicadas con el mismo valor de clave principal (o más exactamente, con el mismo [clave de clasificación](mergetree.md) valor). - -La desduplicación de datos solo se produce durante una fusión. La fusión ocurre en segundo plano en un momento desconocido, por lo que no puede planificarla. Algunos de los datos pueden permanecer sin procesar. Aunque puede ejecutar una fusión no programada utilizando el `OPTIMIZE` consulta, no cuente con usarlo, porque el `OPTIMIZE` consulta leerá y escribirá una gran cantidad de datos. - -Así, `ReplacingMergeTree` es adecuado para borrar datos duplicados en segundo plano para ahorrar espacio, pero no garantiza la ausencia de duplicados. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = ReplacingMergeTree([ver]) -[PARTITION BY expr] -[ORDER BY expr] -[PRIMARY KEY expr] -[SAMPLE BY expr] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros de solicitud, consulte [descripción de la solicitud](../../../sql-reference/statements/create.md). - -**ReplacingMergeTree Parámetros** - -- `ver` — column with version. Type `UInt*`, `Date` o `DateTime`. Parámetro opcional. - - Al fusionar, `ReplacingMergeTree` de todas las filas con la misma clave primaria deja solo una: - - - Último en la selección, si `ver` no establecido. - - Con la versión máxima, si `ver` indicado. - -**Cláusulas de consulta** - -Al crear un `ReplacingMergeTree` mesa de la misma [clausula](mergetree.md) se requieren, como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No use este método en proyectos nuevos y, si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] ReplacingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [ver]) -``` - -Todos los parámetros excepto `ver` el mismo significado que en `MergeTree`. - -- `ver` - columna con la versión. Parámetro opcional. Para una descripción, vea el texto anterior. - -
- -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/replacingmergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/replication.md b/docs/es/engines/table-engines/mergetree-family/replication.md deleted file mode 100644 index 505f5223800..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/replication.md +++ /dev/null @@ -1,218 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: "Replicaci\xF3n de datos" ---- - -# Replicación de datos {#table_engines-replication} - -La replicación solo se admite para tablas de la familia MergeTree: - -- ReplicatedMergeTree -- ReplicatedSummingMergeTree -- ReplicatedReplacingMergeTree -- ReplicatedAggregatingMergeTree -- ReplicatedCollapsingMergeTree -- ReplicatedVersionedCollapsingMergetree -- ReplicatedGraphiteMergeTree - -La replicación funciona a nivel de una tabla individual, no de todo el servidor. Un servidor puede almacenar tablas replicadas y no replicadas al mismo tiempo. - -La replicación no depende de la fragmentación. Cada fragmento tiene su propia replicación independiente. - -Datos comprimidos para `INSERT` y `ALTER` se replica (para obtener más información, consulte la documentación para [ALTER](../../../sql-reference/statements/alter.md#query_language_queries_alter)). - -`CREATE`, `DROP`, `ATTACH`, `DETACH` y `RENAME` las consultas se ejecutan en un único servidor y no se replican: - -- El `CREATE TABLE` query crea una nueva tabla replicable en el servidor donde se ejecuta la consulta. Si esta tabla ya existe en otros servidores, agrega una nueva réplica. -- El `DROP TABLE` query elimina la réplica ubicada en el servidor donde se ejecuta la consulta. -- El `RENAME` query cambia el nombre de la tabla en una de las réplicas. En otras palabras, las tablas replicadas pueden tener diferentes nombres en diferentes réplicas. - -Uso de ClickHouse [Apache ZooKeeper](https://zookeeper.apache.org) para almacenar metainformación de réplicas. Utilice ZooKeeper versión 3.4.5 o posterior. - -Para utilizar la replicación, establezca los parámetros [Zookeeper](../../../operations/server-configuration-parameters/settings.md#server-settings_zookeeper) sección de configuración del servidor. - -!!! attention "Atención" - No descuides la configuración de seguridad. ClickHouse soporta el `digest` [Esquema de ACL](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) del subsistema de seguridad ZooKeeper. - -Ejemplo de configuración de las direcciones del clúster ZooKeeper: - -``` xml - - - example1 - 2181 - - - example2 - 2181 - - - example3 - 2181 - - -``` - -Puede especificar cualquier clúster ZooKeeper existente y el sistema utilizará un directorio en él para sus propios datos (el directorio se especifica al crear una tabla replicable). - -Si ZooKeeper no está establecido en el archivo de configuración, no puede crear tablas replicadas y las tablas replicadas existentes serán de solo lectura. - -ZooKeeper no se utiliza en `SELECT` consultas porque la replicación no afecta al rendimiento de `SELECT` y las consultas se ejecutan tan rápido como lo hacen para las tablas no replicadas. Al consultar tablas replicadas distribuidas, el comportamiento de ClickHouse se controla mediante la configuración [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) y [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). - -Para cada `INSERT` consulta, aproximadamente diez entradas se agregan a ZooKeeper a través de varias transacciones. (Para ser más precisos, esto es para cada bloque de datos insertado; una consulta INSERT contiene un bloque o un bloque por `max_insert_block_size = 1048576` filas.) Esto conduce a latencias ligeramente más largas para `INSERT` en comparación con las tablas no replicadas. Pero si sigue las recomendaciones para insertar datos en lotes de no más de uno `INSERT` por segundo, no crea ningún problema. Todo el clúster ClickHouse utilizado para coordinar un clúster ZooKeeper tiene un total de varios cientos `INSERTs` por segundo. El rendimiento en las inserciones de datos (el número de filas por segundo) es tan alto como para los datos no replicados. - -Para clústeres muy grandes, puede usar diferentes clústeres de ZooKeeper para diferentes fragmentos. Sin embargo, esto no ha demostrado ser necesario en el Yandex.Clúster Metrica (aproximadamente 300 servidores). - -La replicación es asíncrona y multi-master. `INSERT` consultas (así como `ALTER`) se puede enviar a cualquier servidor disponible. Los datos se insertan en el servidor donde se ejecuta la consulta y, a continuación, se copian a los demás servidores. Debido a que es asincrónico, los datos insertados recientemente aparecen en las otras réplicas con cierta latencia. Si parte de las réplicas no está disponible, los datos se escriben cuando estén disponibles. Si hay una réplica disponible, la latencia es la cantidad de tiempo que tarda en transferir el bloque de datos comprimidos a través de la red. - -De forma predeterminada, una consulta INSERT espera la confirmación de la escritura de los datos de una sola réplica. Si los datos fue correctamente escrito a sólo una réplica y el servidor con esta réplica deja de existir, los datos almacenados se perderán. Para habilitar la confirmación de las escrituras de datos de varias réplicas, utilice `insert_quorum` opcion. - -Cada bloque de datos se escribe atómicamente. La consulta INSERT se divide en bloques hasta `max_insert_block_size = 1048576` filas. En otras palabras, si el `INSERT` consulta tiene menos de 1048576 filas, se hace atómicamente. - -Los bloques de datos se deduplican. Para varias escrituras del mismo bloque de datos (bloques de datos del mismo tamaño que contienen las mismas filas en el mismo orden), el bloque solo se escribe una vez. La razón de esto es en caso de fallas de red cuando la aplicación cliente no sabe si los datos se escribieron en la base de datos, por lo que `INSERT` consulta simplemente se puede repetir. No importa a qué réplica se enviaron los INSERT con datos idénticos. `INSERTs` son idempotentes. Los parámetros de desduplicación son controlados por [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) configuración del servidor. - -Durante la replicación, sólo los datos de origen que se van a insertar se transfieren a través de la red. La transformación de datos adicional (fusión) se coordina y se realiza en todas las réplicas de la misma manera. Esto minimiza el uso de la red, lo que significa que la replicación funciona bien cuando las réplicas residen en centros de datos diferentes. (Tenga en cuenta que la duplicación de datos en diferentes centros de datos es el objetivo principal de la replicación.) - -Puede tener cualquier número de réplicas de los mismos datos. El Yandex.Metrica utiliza doble replicación en producción. Cada servidor utiliza RAID-5 o RAID-6, y RAID-10 en algunos casos. Esta es una solución relativamente confiable y conveniente. - -El sistema supervisa la sincronicidad de los datos en las réplicas y puede recuperarse después de un fallo. La conmutación por error es automática (para pequeñas diferencias en los datos) o semiautomática (cuando los datos difieren demasiado, lo que puede indicar un error de configuración). - -## Creación de tablas replicadas {#creating-replicated-tables} - -El `Replicated` prefijo se agrega al nombre del motor de tabla. Por ejemplo:`ReplicatedMergeTree`. - -**Replicated\*MergeTree parámetros** - -- `zoo_path` — The path to the table in ZooKeeper. -- `replica_name` — The replica name in ZooKeeper. - -Ejemplo: - -``` sql -CREATE TABLE table_name -( - EventDate DateTime, - CounterID UInt32, - UserID UInt32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}') -PARTITION BY toYYYYMM(EventDate) -ORDER BY (CounterID, EventDate, intHash32(UserID)) -SAMPLE BY intHash32(UserID) -``` - -
- -Ejemplo en sintaxis obsoleta - -``` sql -CREATE TABLE table_name -( - EventDate DateTime, - CounterID UInt32, - UserID UInt32 -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/table_name', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) -``` - -
- -Como muestra el ejemplo, estos parámetros pueden contener sustituciones entre llaves. Los valores sustituidos se toman de la ‘macros’ sección del archivo de configuración. Ejemplo: - -``` xml - - 05 - 02 - example05-02-1.yandex.ru - -``` - -La ruta de acceso a la tabla en ZooKeeper debe ser única para cada tabla replicada. Las tablas en diferentes fragmentos deben tener rutas diferentes. -En este caso, la ruta consta de las siguientes partes: - -`/clickhouse/tables/` es el prefijo común. Recomendamos usar exactamente este. - -`{layer}-{shard}` es el identificador de fragmento. En este ejemplo consta de dos partes, ya que el Yandex.Metrica clúster utiliza sharding de dos niveles. Para la mayoría de las tareas, puede dejar solo la sustitución {shard}, que se expandirá al identificador de fragmento. - -`table_name` es el nombre del nodo de la tabla en ZooKeeper. Es una buena idea hacerlo igual que el nombre de la tabla. Se define explícitamente, porque a diferencia del nombre de la tabla, no cambia después de una consulta RENAME. -*HINT*: podría agregar un nombre de base de datos delante de `table_name` También. Nivel de Cifrado WEP `db_name.table_name` - -El nombre de réplica identifica diferentes réplicas de la misma tabla. Puede usar el nombre del servidor para esto, como en el ejemplo. El nombre solo tiene que ser único dentro de cada fragmento. - -Puede definir los parámetros explícitamente en lugar de utilizar sustituciones. Esto podría ser conveniente para probar y para configurar clústeres pequeños. Sin embargo, no puede usar consultas DDL distribuidas (`ON CLUSTER` en este caso. - -Cuando se trabaja con clústeres grandes, se recomienda utilizar sustituciones porque reducen la probabilidad de error. - -Ejecute el `CREATE TABLE` consulta en cada réplica. Esta consulta crea una nueva tabla replicada o agrega una nueva réplica a una existente. - -Si agrega una nueva réplica después de que la tabla ya contenga algunos datos en otras réplicas, los datos se copiarán de las otras réplicas a la nueva después de ejecutar la consulta. En otras palabras, la nueva réplica se sincroniza con las demás. - -Para eliminar una réplica, ejecute `DROP TABLE`. However, only one replica is deleted – the one that resides on the server where you run the query. - -## Recuperación después de fallos {#recovery-after-failures} - -Si ZooKeeper no está disponible cuando se inicia un servidor, las tablas replicadas cambian al modo de solo lectura. El sistema intenta conectarse periódicamente a ZooKeeper. - -Si ZooKeeper no está disponible durante un `INSERT`, o se produce un error al interactuar con ZooKeeper, se produce una excepción. - -Después de conectarse a ZooKeeper, el sistema comprueba si el conjunto de datos en el sistema de archivos local coincide con el conjunto de datos esperado (ZooKeeper almacena esta información). Si hay incoherencias menores, el sistema las resuelve sincronizando datos con las réplicas. - -Si el sistema detecta partes de datos rotas (con un tamaño incorrecto de archivos) o partes no reconocidas (partes escritas en el sistema de archivos pero no grabadas en ZooKeeper), las mueve al `detached` subdirectorio (no se eliminan). Las piezas que faltan se copian de las réplicas. - -Tenga en cuenta que ClickHouse no realiza ninguna acción destructiva, como eliminar automáticamente una gran cantidad de datos. - -Cuando el servidor se inicia (o establece una nueva sesión con ZooKeeper), solo verifica la cantidad y el tamaño de todos los archivos. Si los tamaños de los archivos coinciden pero los bytes se han cambiado en algún punto intermedio, esto no se detecta inmediatamente, sino solo cuando se intenta leer los datos `SELECT` consulta. La consulta produce una excepción sobre una suma de comprobación no coincidente o el tamaño de un bloque comprimido. En este caso, las partes de datos se agregan a la cola de verificación y se copian de las réplicas si es necesario. - -Si el conjunto local de datos difiere demasiado del esperado, se activa un mecanismo de seguridad. El servidor ingresa esto en el registro y se niega a iniciarse. La razón de esto es que este caso puede indicar un error de configuración, como si una réplica en un fragmento se configurara accidentalmente como una réplica en un fragmento diferente. Sin embargo, los umbrales para este mecanismo se establecen bastante bajos, y esta situación puede ocurrir durante la recuperación de falla normal. En este caso, los datos se restauran semiautomáticamente, mediante “pushing a button”. - -Para iniciar la recuperación, cree el nodo `/path_to_table/replica_name/flags/force_restore_data` en ZooKeeper con cualquier contenido, o ejecute el comando para restaurar todas las tablas replicadas: - -``` bash -sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data -``` - -A continuación, reinicie el servidor. Al iniciar, el servidor elimina estos indicadores e inicia la recuperación. - -## Recuperación después de la pérdida completa de datos {#recovery-after-complete-data-loss} - -Si todos los datos y metadatos desaparecieron de uno de los servidores, siga estos pasos para la recuperación: - -1. Instale ClickHouse en el servidor. Defina correctamente las sustituciones en el archivo de configuración que contiene el identificador de fragmento y las réplicas, si las usa. -2. Si tenía tablas no duplicadas que deben duplicarse manualmente en los servidores, copie sus datos desde una réplica (en el directorio `/var/lib/clickhouse/data/db_name/table_name/`). -3. Copiar definiciones de tablas ubicadas en `/var/lib/clickhouse/metadata/` de una réplica. Si un identificador de fragmento o réplica se define explícitamente en las definiciones de tabla, corríjalo para que corresponda a esta réplica. (Como alternativa, inicie el servidor y `ATTACH TABLE` consultas que deberían haber estado en el .sql archivos en `/var/lib/clickhouse/metadata/`.) -4. Para iniciar la recuperación, cree el nodo ZooKeeper `/path_to_table/replica_name/flags/force_restore_data` con cualquier contenido o ejecute el comando para restaurar todas las tablas replicadas: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` - -Luego inicie el servidor (reinicie, si ya se está ejecutando). Los datos se descargarán de las réplicas. - -Una opción de recuperación alternativa es eliminar información sobre la réplica perdida de ZooKeeper (`/path_to_table/replica_name`), luego vuelva a crear la réplica como se describe en “[Creación de tablas replicadas](#creating-replicated-tables)”. - -No hay restricción en el ancho de banda de la red durante la recuperación. Tenga esto en cuenta si está restaurando muchas réplicas a la vez. - -## La conversión de MergeTree a ReplicatedMergeTree {#converting-from-mergetree-to-replicatedmergetree} - -Usamos el término `MergeTree` para referirse a todos los motores de mesa en el `MergeTree family`, lo mismo que para `ReplicatedMergeTree`. - -Si usted tenía un `MergeTree` tabla replicada manualmente, puede convertirla en una tabla replicada. Es posible que tenga que hacer esto si ya ha recopilado una gran cantidad de datos `MergeTree` y ahora desea habilitar la replicación. - -Si los datos difieren en varias réplicas, primero sincronícelos o elimínelos en todas las réplicas, excepto en una. - -Cambie el nombre de la tabla MergeTree existente y, a continuación, cree un `ReplicatedMergeTree` mesa con el antiguo nombre. -Mueva los datos de la tabla antigua a la `detached` subdirectorio dentro del directorio con los nuevos datos de la tabla (`/var/lib/clickhouse/data/db_name/table_name/`). -Luego ejecuta `ALTER TABLE ATTACH PARTITION` en una de las réplicas para agregar estas partes de datos al conjunto de trabajo. - -## La conversión de ReplicatedMergeTree a MergeTree {#converting-from-replicatedmergetree-to-mergetree} - -Cree una tabla MergeTree con un nombre diferente. Mueva todos los datos del directorio con el `ReplicatedMergeTree` datos de la tabla al directorio de datos de la nueva tabla. A continuación, elimine el `ReplicatedMergeTree` y reinicie el servidor. - -Si desea deshacerse de un `ReplicatedMergeTree` sin iniciar el servidor: - -- Eliminar el correspondiente `.sql` archivo en el directorio de metadatos (`/var/lib/clickhouse/metadata/`). -- Eliminar la ruta correspondiente en ZooKeeper (`/path_to_table/replica_name`). - -Después de esto, puede iniciar el servidor, crear un `MergeTree` tabla, mueva los datos a su directorio y, a continuación, reinicie el servidor. - -## Recuperación cuando se pierden o se dañan los metadatos del clúster Zookeeper {#recovery-when-metadata-in-the-zookeeper-cluster-is-lost-or-damaged} - -Si los datos de ZooKeeper se perdieron o se dañaron, puede guardar los datos moviéndolos a una tabla no duplicada como se describió anteriormente. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/replication/) diff --git a/docs/es/engines/table-engines/mergetree-family/summingmergetree.md b/docs/es/engines/table-engines/mergetree-family/summingmergetree.md deleted file mode 100644 index 3ae9a1515c0..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/summingmergetree.md +++ /dev/null @@ -1,141 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 34 -toc_title: SummingMergeTree ---- - -# SummingMergeTree {#summingmergetree} - -El motor hereda de [Método de codificación de datos:](mergetree.md#table_engines-mergetree). La diferencia es que al fusionar partes de datos para `SummingMergeTree` ClickHouse reemplaza todas las filas con la misma clave primaria (o más exactamente, con la misma [clave de clasificación](mergetree.md)) con una fila que contiene valores resumidos para las columnas con el tipo de datos numérico. Si la clave de ordenación está compuesta de manera que un solo valor de clave corresponde a un gran número de filas, esto reduce significativamente el volumen de almacenamiento y acelera la selección de datos. - -Recomendamos usar el motor junto con `MergeTree`. Almacenar datos completos en `MergeTree` mesa, y el uso `SummingMergeTree` para el almacenamiento de datos agregados, por ejemplo, al preparar informes. Tal enfoque evitará que pierda datos valiosos debido a una clave primaria compuesta incorrectamente. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = SummingMergeTree([columns]) -[PARTITION BY expr] -[ORDER BY expr] -[SAMPLE BY expr] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros de solicitud, consulte [descripción de la solicitud](../../../sql-reference/statements/create.md). - -**Parámetros de SummingMergeTree** - -- `columns` - una tupla con los nombres de las columnas donde se resumirán los valores. Parámetro opcional. - Las columnas deben ser de tipo numérico y no deben estar en la clave principal. - - Si `columns` no especificado, ClickHouse resume los valores de todas las columnas con un tipo de datos numérico que no están en la clave principal. - -**Cláusulas de consulta** - -Al crear un `SummingMergeTree` mesa de la misma [clausula](mergetree.md) se requieren, como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No use este método en proyectos nuevos y, si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] SummingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [columns]) -``` - -Todos los parámetros excepto `columns` el mismo significado que en `MergeTree`. - -- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above. - -
- -## Ejemplo de uso {#usage-example} - -Considere la siguiente tabla: - -``` sql -CREATE TABLE summtt -( - key UInt32, - value UInt32 -) -ENGINE = SummingMergeTree() -ORDER BY key -``` - -Insertar datos: - -``` sql -INSERT INTO summtt Values(1,1),(1,2),(2,1) -``` - -ClickHouse puede sumar todas las filas no completamente ([ver abajo](#data-processing)), entonces usamos una función agregada `sum` y `GROUP BY` cláusula en la consulta. - -``` sql -SELECT key, sum(value) FROM summtt GROUP BY key -``` - -``` text -┌─key─┬─sum(value)─┐ -│ 2 │ 1 │ -│ 1 │ 3 │ -└─────┴────────────┘ -``` - -## Procesamiento de datos {#data-processing} - -Cuando los datos se insertan en una tabla, se guardan tal cual. ClickHouse combina las partes insertadas de los datos periódicamente y esto es cuando las filas con la misma clave principal se suman y se reemplazan con una para cada parte resultante de los datos. - -ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) una función agregada [resumir()](../../../sql-reference/aggregate-functions/reference.md#agg_function-sum) y `GROUP BY` cláusula se debe utilizar en una consulta como se describe en el ejemplo anterior. - -### Reglas comunes para la suma {#common-rules-for-summation} - -Se resumen los valores de las columnas con el tipo de datos numérico. El conjunto de columnas está definido por el parámetro `columns`. - -Si los valores eran 0 en todas las columnas para la suma, se elimina la fila. - -Si la columna no está en la clave principal y no se resume, se selecciona un valor arbitrario entre los existentes. - -Los valores no se resumen para las columnas de la clave principal. - -### La suma en las columnas de función agregada {#the-summation-in-the-aggregatefunction-columns} - -Para columnas de [Tipo AggregateFunction](../../../sql-reference/data-types/aggregatefunction.md) ClickHouse se comporta como [AgregaciónMergeTree](aggregatingmergetree.md) agregación del motor según la función. - -### Estructuras anidadas {#nested-structures} - -La tabla puede tener estructuras de datos anidadas que se procesan de una manera especial. - -Si el nombre de una tabla anidada termina con `Map` y contiene al menos dos columnas que cumplen los siguientes criterios: - -- la primera columna es numérica `(*Int*, Date, DateTime)` o una cadena `(String, FixedString)`, vamos a llamarlo `key`, -- las otras columnas son aritméticas `(*Int*, Float32/64)`, vamos a llamarlo `(values...)`, - -entonces esta tabla anidada se interpreta como una asignación de `key => (values...)`, y al fusionar sus filas, los elementos de dos conjuntos de datos se fusionan por `key` con una suma de los correspondientes `(values...)`. - -Ejemplos: - -``` text -[(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] -[(1, 100)] + [(1, 150)] -> [(1, 250)] -[(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)] -[(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] -``` - -Al solicitar datos, utilice el [sumMap(clave, valor)](../../../sql-reference/aggregate-functions/reference.md) función para la agregación de `Map`. - -Para la estructura de datos anidados, no necesita especificar sus columnas en la tupla de columnas para la suma. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/summingmergetree/) diff --git a/docs/es/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/es/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md deleted file mode 100644 index d69bfe9440e..00000000000 --- a/docs/es/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ /dev/null @@ -1,238 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: VersionedCollapsingMergeTree ---- - -# VersionedCollapsingMergeTree {#versionedcollapsingmergetree} - -Este motor: - -- Permite la escritura rápida de estados de objetos que cambian continuamente. -- Elimina los estados de objetos antiguos en segundo plano. Esto reduce significativamente el volumen de almacenamiento. - -Vea la sección [Derrumbar](#table_engines_versionedcollapsingmergetree) para más detalles. - -El motor hereda de [Método de codificación de datos:](mergetree.md#table_engines-mergetree) y agrega la lógica para colapsar filas al algoritmo para fusionar partes de datos. `VersionedCollapsingMergeTree` tiene el mismo propósito que [ColapsarMergeTree](collapsingmergetree.md) pero usa un algoritmo de colapso diferente que permite insertar los datos en cualquier orden con múltiples hilos. En particular, el `Version` columna ayuda a contraer las filas correctamente, incluso si se insertan en el orden incorrecto. En contraste, `CollapsingMergeTree` sólo permite la inserción estrictamente consecutiva. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE = VersionedCollapsingMergeTree(sign, version) -[PARTITION BY expr] -[ORDER BY expr] -[SAMPLE BY expr] -[SETTINGS name=value, ...] -``` - -Para obtener una descripción de los parámetros de consulta, consulte [descripción de la consulta](../../../sql-reference/statements/create.md). - -**Parámetros del motor** - -``` sql -VersionedCollapsingMergeTree(sign, version) -``` - -- `sign` — Name of the column with the type of row: `1` es una “state” fila, `-1` es una “cancel” fila. - - El tipo de datos de columna debe ser `Int8`. - -- `version` — Name of the column with the version of the object state. - - El tipo de datos de columna debe ser `UInt*`. - -**Cláusulas de consulta** - -Al crear un `VersionedCollapsingMergeTree` mesa, la misma [clausula](mergetree.md) se requieren como al crear un `MergeTree` tabla. - -
- -Método obsoleto para crear una tabla - -!!! attention "Atención" - No utilice este método en nuevos proyectos. Si es posible, cambie los proyectos antiguos al método descrito anteriormente. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) ENGINE [=] VersionedCollapsingMergeTree(date-column [, samp#table_engines_versionedcollapsingmergetreeling_expression], (primary, key), index_granularity, sign, version) -``` - -Todos los parámetros excepto `sign` y `version` el mismo significado que en `MergeTree`. - -- `sign` — Name of the column with the type of row: `1` es una “state” fila, `-1` es una “cancel” fila. - - Column Data Type — `Int8`. - -- `version` — Name of the column with the version of the object state. - - El tipo de datos de columna debe ser `UInt*`. - -
- -## Derrumbar {#table_engines_versionedcollapsingmergetree} - -### Datos {#data} - -Considere una situación en la que necesite guardar datos que cambien continuamente para algún objeto. Es razonable tener una fila para un objeto y actualizar la fila siempre que haya cambios. Sin embargo, la operación de actualización es costosa y lenta para un DBMS porque requiere volver a escribir los datos en el almacenamiento. La actualización no es aceptable si necesita escribir datos rápidamente, pero puede escribir los cambios en un objeto secuencialmente de la siguiente manera. - -Utilice el `Sign` columna al escribir la fila. Si `Sign = 1` significa que la fila es un estado de un objeto (llamémoslo el “state” fila). Si `Sign = -1` indica la cancelación del estado de un objeto con los mismos atributos (llamémoslo el “cancel” fila). También use el `Version` columna, que debe identificar cada estado de un objeto con un número separado. - -Por ejemplo, queremos calcular cuántas páginas visitaron los usuarios en algún sitio y cuánto tiempo estuvieron allí. En algún momento escribimos la siguiente fila con el estado de la actividad del usuario: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ 1 | -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -En algún momento después registramos el cambio de actividad del usuario y lo escribimos con las siguientes dos filas. - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ 1 | -│ 4324182021466249494 │ 6 │ 185 │ 1 │ 2 | -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -La primera fila cancela el estado anterior del objeto (usuario). Debe copiar todos los campos del estado cancelado excepto `Sign`. - -La segunda fila contiene el estado actual. - -Debido a que solo necesitamos el último estado de actividad del usuario, las filas - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ 1 | -│ 4324182021466249494 │ 5 │ 146 │ -1 │ 1 | -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -se puede eliminar, colapsando el estado no válido (antiguo) del objeto. `VersionedCollapsingMergeTree` hace esto mientras fusiona las partes de datos. - -Para averiguar por qué necesitamos dos filas para cada cambio, vea [Algoritmo](#table_engines-versionedcollapsingmergetree-algorithm). - -**Notas sobre el uso** - -1. El programa que escribe los datos debe recordar el estado de un objeto para cancelarlo. El “cancel” cadena debe ser una copia de la “state” con lo opuesto `Sign`. Esto aumenta el tamaño inicial de almacenamiento, pero permite escribir los datos rápidamente. -2. Las matrices de largo crecimiento en columnas reducen la eficiencia del motor debido a la carga para escribir. Cuanto más sencillos sean los datos, mejor será la eficiencia. -3. `SELECT` Los resultados dependen en gran medida de la coherencia del historial de cambios de objetos. Sea preciso al preparar los datos para insertarlos. Puede obtener resultados impredecibles con datos incoherentes, como valores negativos para métricas no negativas, como la profundidad de la sesión. - -### Algoritmo {#table_engines-versionedcollapsingmergetree-algorithm} - -Cuando ClickHouse combina partes de datos, elimina cada par de filas que tienen la misma clave principal y versión y diferentes `Sign`. El orden de las filas no importa. - -Cuando ClickHouse inserta datos, ordena filas por la clave principal. Si el `Version` la columna no está en la clave principal, ClickHouse la agrega a la clave principal implícitamente como el último campo y la usa para ordenar. - -## Selección de datos {#selecting-data} - -ClickHouse no garantiza que todas las filas con la misma clave principal estén en la misma parte de datos resultante o incluso en el mismo servidor físico. Esto es cierto tanto para escribir los datos como para la posterior fusión de las partes de datos. Además, ClickHouse procesa `SELECT` consultas con múltiples subprocesos, y no puede predecir el orden de las filas en el resultado. Esto significa que la agregación es necesaria si hay una necesidad de obtener completamente “collapsed” datos de un `VersionedCollapsingMergeTree` tabla. - -Para finalizar el colapso, escriba una consulta con un `GROUP BY` cláusula y funciones agregadas que representan el signo. Por ejemplo, para calcular la cantidad, use `sum(Sign)` en lugar de `count()`. Para calcular la suma de algo, use `sum(Sign * x)` en lugar de `sum(x)` y agregar `HAVING sum(Sign) > 0`. - -Los agregados `count`, `sum` y `avg` se puede calcular de esta manera. El agregado `uniq` se puede calcular si un objeto tiene al menos un estado no colapsado. Los agregados `min` y `max` no se puede calcular porque `VersionedCollapsingMergeTree` no guarda el historial de valores de estados colapsados. - -Si necesita extraer los datos con “collapsing” pero sin agregación (por ejemplo, para verificar si hay filas presentes cuyos valores más nuevos coinciden con ciertas condiciones), puede usar el `FINAL` modificador para el `FROM` clausula. Este enfoque es ineficiente y no debe usarse con tablas grandes. - -## Ejemplo de uso {#example-of-use} - -Datos de ejemplo: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ 1 | -│ 4324182021466249494 │ 5 │ 146 │ -1 │ 1 | -│ 4324182021466249494 │ 6 │ 185 │ 1 │ 2 | -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -Creación de la tabla: - -``` sql -CREATE TABLE UAct -( - UserID UInt64, - PageViews UInt8, - Duration UInt8, - Sign Int8, - Version UInt8 -) -ENGINE = VersionedCollapsingMergeTree(Sign, Version) -ORDER BY UserID -``` - -Insertar los datos: - -``` sql -INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1, 1) -``` - -``` sql -INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1, 1),(4324182021466249494, 6, 185, 1, 2) -``` - -Usamos dos `INSERT` consultas para crear dos partes de datos diferentes. Si insertamos los datos con una sola consulta, ClickHouse crea una parte de datos y nunca realizará ninguna fusión. - -Obtener los datos: - -``` sql -SELECT * FROM UAct -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ 1 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ 1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ 2 │ -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -¿Qué vemos aquí y dónde están las partes colapsadas? -Creamos dos partes de datos usando dos `INSERT` consulta. El `SELECT` la consulta se realizó en dos subprocesos, y el resultado es un orden aleatorio de filas. -No se produjo el colapso porque las partes de datos aún no se han fusionado. ClickHouse fusiona partes de datos en un punto desconocido en el tiempo que no podemos predecir. - -Es por eso que necesitamos agregación: - -``` sql -SELECT - UserID, - sum(PageViews * Sign) AS PageViews, - sum(Duration * Sign) AS Duration, - Version -FROM UAct -GROUP BY UserID, Version -HAVING sum(Sign) > 0 -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Version─┐ -│ 4324182021466249494 │ 6 │ 185 │ 2 │ -└─────────────────────┴───────────┴──────────┴─────────┘ -``` - -Si no necesitamos agregación y queremos forzar el colapso, podemos usar el `FINAL` modificador para el `FROM` clausula. - -``` sql -SELECT * FROM UAct FINAL -``` - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┬─Version─┐ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ 2 │ -└─────────────────────┴───────────┴──────────┴──────┴─────────┘ -``` - -Esta es una forma muy ineficiente de seleccionar datos. No lo use para mesas grandes. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/versionedcollapsingmergetree/) diff --git a/docs/es/engines/table-engines/special/buffer.md b/docs/es/engines/table-engines/special/buffer.md deleted file mode 100644 index b3a26ff356a..00000000000 --- a/docs/es/engines/table-engines/special/buffer.md +++ /dev/null @@ -1,71 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: "B\xFAfer" ---- - -# Búfer {#buffer} - -Almacena los datos para escribir en la memoria RAM, enjuagándolos periódicamente a otra tabla. Durante la operación de lectura, los datos se leen desde el búfer y la otra tabla simultáneamente. - -``` sql -Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) -``` - -Parámetros del motor: - -- `database` – Database name. Instead of the database name, you can use a constant expression that returns a string. -- `table` – Table to flush data to. -- `num_layers` – Parallelism layer. Physically, the table will be represented as `num_layers` de búferes independientes. Valor recomendado: 16. -- `min_time`, `max_time`, `min_rows`, `max_rows`, `min_bytes`, y `max_bytes` – Conditions for flushing data from the buffer. - -Los datos se vacían del búfer y se escriben en la tabla de destino si `min*` condiciones o al menos una `max*` condición se cumplen. - -- `min_time`, `max_time` – Condition for the time in seconds from the moment of the first write to the buffer. -- `min_rows`, `max_rows` – Condition for the number of rows in the buffer. -- `min_bytes`, `max_bytes` – Condition for the number of bytes in the buffer. - -Durante la operación de escritura, los datos se insertan en un `num_layers` número de búferes aleatorios. O bien, si la parte de datos para insertar es lo suficientemente grande (mayor que `max_rows` o `max_bytes`), se escribe directamente en la tabla de destino, omitiendo el búfer. - -Las condiciones para el lavado de los datos se calculan por separado para cada uno de los `num_layers` búfer. Por ejemplo, si `num_layers = 16` y `max_bytes = 100000000`, el consumo máximo de RAM es de 1.6 GB. - -Ejemplo: - -``` sql -CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) -``` - -Creación de un ‘merge.hits_buffer’ mesa con la misma estructura que ‘merge.hits’ y usando el motor Buffer. Al escribir en esta tabla, los datos se almacenan en la memoria RAM y ‘merge.hits’ tabla. Se crean 16 búferes. Los datos de cada uno de ellos se vacían si han pasado 100 segundos o se han escrito un millón de filas o se han escrito 100 MB de datos; o si simultáneamente han pasado 10 segundos y se han escrito 10.000 filas y 10 MB de datos. Por ejemplo, si solo se ha escrito una fila, después de 100 segundos se vaciará, pase lo que pase. Pero si se han escrito muchas filas, los datos se vaciarán antes. - -Cuando se detiene el servidor, con DROP TABLE o DETACH TABLE, los datos del búfer también se vacían a la tabla de destino. - -Puede establecer cadenas vacías entre comillas simples para la base de datos y el nombre de la tabla. Esto indica la ausencia de una tabla de destino. En este caso, cuando se alcanzan las condiciones de descarga de datos, el búfer simplemente se borra. Esto puede ser útil para mantener una ventana de datos en la memoria. - -Al leer desde una tabla de búfer, los datos se procesan tanto desde el búfer como desde la tabla de destino (si hay uno). -Tenga en cuenta que las tablas Buffer no admiten un índice. En otras palabras, los datos del búfer se analizan por completo, lo que puede ser lento para los búferes grandes. (Para los datos de una tabla subordinada, se utilizará el índice que admite.) - -Si el conjunto de columnas de la tabla Buffer no coincide con el conjunto de columnas de una tabla subordinada, se inserta un subconjunto de columnas que existen en ambas tablas. - -Si los tipos no coinciden con una de las columnas de la tabla Búfer y una tabla subordinada, se escribe un mensaje de error en el registro del servidor y se borra el búfer. -Lo mismo sucede si la tabla subordinada no existe cuando se vacía el búfer. - -Si necesita ejecutar ALTER para una tabla subordinada y la tabla de búfer, se recomienda eliminar primero la tabla de búfer, ejecutar ALTER para la tabla subordinada y, a continuación, crear la tabla de búfer de nuevo. - -Si el servidor se reinicia de forma anormal, se pierden los datos del búfer. - -FINAL y SAMPLE no funcionan correctamente para las tablas Buffer. Estas condiciones se pasan a la tabla de destino, pero no se utilizan para procesar datos en el búfer. Si se requieren estas características, recomendamos usar solo la tabla Buffer para escribir, mientras lee desde la tabla de destino. - -Al agregar datos a un búfer, uno de los búferes está bloqueado. Esto provoca retrasos si se realiza una operación de lectura simultáneamente desde la tabla. - -Los datos que se insertan en una tabla de búfer pueden terminar en la tabla subordinada en un orden diferente y en bloques diferentes. Debido a esto, una tabla Buffer es difícil de usar para escribir en un CollapsingMergeTree correctamente. Para evitar problemas, puede establecer ‘num_layers’ a 1. - -Si se replica la tabla de destino, se pierden algunas características esperadas de las tablas replicadas al escribir en una tabla de búfer. Los cambios aleatorios en el orden de las filas y los tamaños de las partes de datos hacen que la desduplicación de datos deje de funcionar, lo que significa que no es posible tener un ‘exactly once’ escribir en tablas replicadas. - -Debido a estas desventajas, solo podemos recomendar el uso de una tabla Buffer en casos raros. - -Una tabla de búfer se usa cuando se reciben demasiados INSERT de un gran número de servidores durante una unidad de tiempo y los datos no se pueden almacenar en búfer antes de la inserción, lo que significa que los INSERT no pueden ejecutarse lo suficientemente rápido. - -Tenga en cuenta que no tiene sentido insertar datos una fila a la vez, incluso para las tablas de búfer. Esto solo producirá una velocidad de unos pocos miles de filas por segundo, mientras que la inserción de bloques de datos más grandes puede producir más de un millón de filas por segundo (consulte la sección “Performance”). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/buffer/) diff --git a/docs/es/engines/table-engines/special/dictionary.md b/docs/es/engines/table-engines/special/dictionary.md deleted file mode 100644 index 6d9136a6a23..00000000000 --- a/docs/es/engines/table-engines/special/dictionary.md +++ /dev/null @@ -1,97 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 35 -toc_title: Diccionario ---- - -# Diccionario {#dictionary} - -El `Dictionary` el motor muestra el [diccionario](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) datos como una tabla ClickHouse. - -Como ejemplo, considere un diccionario de `products` con la siguiente configuración: - -``` xml - - - products - - -
products
- DSN=some-db-server - - - - 300 - 360 - - - - - - - product_id - - - title - String - - - - - -``` - -Consultar los datos del diccionario: - -``` sql -SELECT - name, - type, - key, - attribute.names, - attribute.types, - bytes_allocated, - element_count, - source -FROM system.dictionaries -WHERE name = 'products' -``` - -``` text -┌─name─────┬─type─┬─key────┬─attribute.names─┬─attribute.types─┬─bytes_allocated─┬─element_count─┬─source──────────┐ -│ products │ Flat │ UInt64 │ ['title'] │ ['String'] │ 23065376 │ 175032 │ ODBC: .products │ -└──────────┴──────┴────────┴─────────────────┴─────────────────┴─────────────────┴───────────────┴─────────────────┘ -``` - -Puede usar el [dictGet\*](../../../sql-reference/functions/ext-dict-functions.md#ext_dict_functions) función para obtener los datos del diccionario en este formato. - -Esta vista no es útil cuando necesita obtener datos sin procesar o cuando `JOIN` operación. Para estos casos, puede usar el `Dictionary` motor, que muestra los datos del diccionario en una tabla. - -Sintaxis: - -``` sql -CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` -``` - -Ejemplo de uso: - -``` sql -create table products (product_id UInt64, title String) Engine = Dictionary(products); -``` - - Ok - -Echa un vistazo a lo que hay en la mesa. - -``` sql -select * from products limit 1; -``` - -``` text -┌────product_id─┬─title───────────┐ -│ 152689 │ Some item │ -└───────────────┴─────────────────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/dictionary/) diff --git a/docs/es/engines/table-engines/special/distributed.md b/docs/es/engines/table-engines/special/distributed.md deleted file mode 100644 index bac407a651a..00000000000 --- a/docs/es/engines/table-engines/special/distributed.md +++ /dev/null @@ -1,152 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 33 -toc_title: Distribuido ---- - -# Distribuido {#distributed} - -**Las tablas con motor distribuido no almacenan ningún dato por sí mismas**, pero permite el procesamiento de consultas distribuidas en varios servidores. -La lectura se paralela automáticamente. Durante una lectura, se utilizan los índices de tabla en servidores remotos, si los hay. - -El motor distribuido acepta parámetros: - -- el nombre del clúster en el archivo de configuración del servidor - -- el nombre de una base de datos remota - -- el nombre de una tabla remota - -- (opcionalmente) clave de fragmentación - -- nombre de política (opcionalmente), se usará para almacenar archivos temporales para el envío asíncrono - - Ver también: - - - `insert_distributed_sync` configuración - - [Método de codificación de datos:](../mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) para los ejemplos - -Ejemplo: - -``` sql -Distributed(logs, default, hits[, sharding_key[, policy_name]]) -``` - -Los datos se leerán desde todos los servidores ‘logs’ clúster, desde el valor predeterminado.tabla de éxitos ubicada en cada servidor del clúster. -Los datos no solo se leen sino que se procesan parcialmente en los servidores remotos (en la medida en que esto sea posible). -Por ejemplo, para una consulta con GROUP BY, los datos se agregarán en servidores remotos y los estados intermedios de las funciones agregadas se enviarán al servidor solicitante. Luego, los datos se agregarán más. - -En lugar del nombre de la base de datos, puede usar una expresión constante que devuelva una cadena. Por ejemplo: currentDatabase(). - -logs – The cluster name in the server's config file. - -Los clústeres se establecen así: - -``` xml - - - - - 1 - - false - - example01-01-1 - 9000 - - - example01-01-2 - 9000 - - - - 2 - false - - example01-02-1 - 9000 - - - example01-02-2 - 1 - 9440 - - - - -``` - -Aquí se define un clúster con el nombre ‘logs’ que consta de dos fragmentos, cada uno de los cuales contiene dos réplicas. -Los fragmentos se refieren a los servidores que contienen diferentes partes de los datos (para leer todos los datos, debe acceder a todos los fragmentos). -Las réplicas están duplicando servidores (para leer todos los datos, puede acceder a los datos en cualquiera de las réplicas). - -Los nombres de clúster no deben contener puntos. - -Los parámetros `host`, `port`, y opcionalmente `user`, `password`, `secure`, `compression` se especifican para cada servidor: -- `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. -- `port` – The TCP port for messenger activity (‘tcp_port’ en la configuración, generalmente establecido en 9000). No lo confundas con http_port. -- `user` – Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section [Derechos de acceso](../../../operations/access-rights.md). -- `password` – The password for connecting to a remote server (not masked). Default value: empty string. -- `secure` - Use ssl para la conexión, por lo general también debe definir `port` = 9440. El servidor debe escuchar en `9440` y tener certificados correctos. -- `compression` - Utilice la compresión de datos. Valor predeterminado: true. - -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) configuración. -Si no se establece la conexión con el servidor, habrá un intento de conectarse con un breve tiempo de espera. Si la conexión falla, se seleccionará la siguiente réplica, y así sucesivamente para todas las réplicas. Si el intento de conexión falló para todas las réplicas, el intento se repetirá de la misma manera, varias veces. -Esto funciona a favor de la resiliencia, pero no proporciona una tolerancia completa a errores: un servidor remoto podría aceptar la conexión, pero podría no funcionar o funcionar mal. - -Puede especificar solo uno de los fragmentos (en este caso, el procesamiento de consultas debe denominarse remoto, en lugar de distribuido) o hasta cualquier número de fragmentos. En cada fragmento, puede especificar entre una y cualquier número de réplicas. Puede especificar un número diferente de réplicas para cada fragmento. - -Puede especificar tantos clústeres como desee en la configuración. - -Para ver los clústeres, utilice el ‘system.clusters’ tabla. - -El motor distribuido permite trabajar con un clúster como un servidor local. Sin embargo, el clúster es inextensible: debe escribir su configuración en el archivo de configuración del servidor (mejor aún, para todos los servidores del clúster). - -The Distributed engine requires writing clusters to the config file. Clusters from the config file are updated on the fly, without restarting the server. If you need to send a query to an unknown set of shards and replicas each time, you don't need to create a Distributed table – use the ‘remote’ función de tabla en su lugar. Vea la sección [Funciones de tabla](../../../sql-reference/table-functions/index.md). - -Hay dos métodos para escribir datos en un clúster: - -Primero, puede definir a qué servidores escribir en qué datos y realizar la escritura directamente en cada fragmento. En otras palabras, realice INSERT en las tablas que la tabla distribuida “looks at”. Esta es la solución más flexible, ya que puede usar cualquier esquema de fragmentación, que podría ser no trivial debido a los requisitos del área temática. Esta es también la solución más óptima ya que los datos se pueden escribir en diferentes fragmentos de forma completamente independiente. - -En segundo lugar, puede realizar INSERT en una tabla distribuida. En este caso, la tabla distribuirá los datos insertados a través de los propios servidores. Para escribir en una tabla distribuida, debe tener un conjunto de claves de fragmentación (el último parámetro). Además, si solo hay un fragmento, la operación de escritura funciona sin especificar la clave de fragmentación, ya que no significa nada en este caso. - -Cada fragmento puede tener un peso definido en el archivo de configuración. Por defecto, el peso es igual a uno. Los datos se distribuyen entre fragmentos en la cantidad proporcional al peso del fragmento. Por ejemplo, si hay dos fragmentos y el primero tiene un peso de 9 mientras que el segundo tiene un peso de 10, el primero se enviará 9 / 19 partes de las filas, y el segundo se enviará 10 / 19. - -Cada fragmento puede tener el ‘internal_replication’ parámetro definido en el archivo de configuración. - -Si este parámetro se establece en ‘true’, la operación de escritura selecciona la primera réplica en buen estado y escribe datos en ella. Utilice esta alternativa si la tabla Distribuida “looks at” tablas replicadas. En otras palabras, si la tabla donde se escribirán los datos los replicará por sí misma. - -Si se establece en ‘false’ (el valor predeterminado), los datos se escriben en todas las réplicas. En esencia, esto significa que la tabla distribuida replica los datos en sí. Esto es peor que usar tablas replicadas, porque no se verifica la consistencia de las réplicas y, con el tiempo, contendrán datos ligeramente diferentes. - -Para seleccionar el fragmento al que se envía una fila de datos, se analiza la expresión de fragmentación y su resto se toma de dividirlo por el peso total de los fragmentos. La fila se envía al fragmento que corresponde al medio intervalo de los restos de ‘prev_weight’ a ‘prev_weights + weight’, donde ‘prev_weights’ es el peso total de los fragmentos con el número más pequeño, y ‘weight’ es el peso de este fragmento. Por ejemplo, si hay dos fragmentos, y el primero tiene un peso de 9 mientras que el segundo tiene un peso de 10, la fila se enviará al primer fragmento para los restos del rango \[0, 9), y al segundo para los restos del rango \[9, 19). - -La expresión de fragmentación puede ser cualquier expresión de constantes y columnas de tabla que devuelva un entero. Por ejemplo, puede usar la expresión ‘rand()’ para la distribución aleatoria de datos, o ‘UserID’ para la distribución por el resto de dividir la ID del usuario (entonces los datos de un solo usuario residirán en un solo fragmento, lo que simplifica la ejecución de IN y JOIN por los usuarios). Si una de las columnas no se distribuye lo suficientemente uniformemente, puede envolverla en una función hash: intHash64(UserID) . - -Un simple recordatorio de la división es una solución limitada para sharding y no siempre es apropiado. Funciona para volúmenes medianos y grandes de datos (docenas de servidores), pero no para volúmenes muy grandes de datos (cientos de servidores o más). En este último caso, use el esquema de fragmentación requerido por el área asunto, en lugar de usar entradas en Tablas distribuidas. - -SELECT queries are sent to all the shards and work regardless of how data is distributed across the shards (they can be distributed completely randomly). When you add a new shard, you don't have to transfer the old data to it. You can write new data with a heavier weight – the data will be distributed slightly unevenly, but queries will work correctly and efficiently. - -Debería preocuparse por el esquema de fragmentación en los siguientes casos: - -- Se utilizan consultas que requieren unir datos (IN o JOIN) mediante una clave específica. Si esta clave fragmenta datos, puede usar IN local o JOIN en lugar de GLOBAL IN o GLOBAL JOIN, que es mucho más eficiente. -- Se usa una gran cantidad de servidores (cientos o más) con una gran cantidad de consultas pequeñas (consultas de clientes individuales: sitios web, anunciantes o socios). Para que las pequeñas consultas no afecten a todo el clúster, tiene sentido ubicar datos para un solo cliente en un solo fragmento. Alternativamente, como lo hemos hecho en Yandex.Metrica, puede configurar sharding de dos niveles: divida todo el clúster en “layers”, donde una capa puede consistir en varios fragmentos. Los datos de un único cliente se encuentran en una sola capa, pero los fragmentos se pueden agregar a una capa según sea necesario y los datos se distribuyen aleatoriamente dentro de ellos. Las tablas distribuidas se crean para cada capa y se crea una única tabla distribuida compartida para consultas globales. - -Los datos se escriben de forma asíncrona. Cuando se inserta en la tabla, el bloque de datos se acaba de escribir en el sistema de archivos local. Los datos se envían a los servidores remotos en segundo plano tan pronto como sea posible. El período de envío de datos está gestionado por el [Distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) y [Distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) configuración. El `Distributed` el motor envía cada archivo con datos insertados por separado, pero puede habilitar el envío por lotes de archivos [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) configuración. Esta configuración mejora el rendimiento del clúster al utilizar mejor los recursos de red y servidor local. Debe comprobar si los datos se envían correctamente comprobando la lista de archivos (datos en espera de ser enviados) en el directorio de la tabla: `/var/lib/clickhouse/data/database/table/`. - -Si el servidor dejó de existir o tuvo un reinicio aproximado (por ejemplo, después de un error de dispositivo) después de un INSERT en una tabla distribuida, es posible que se pierdan los datos insertados. Si se detecta un elemento de datos dañado en el directorio de la tabla, se transfiere al ‘broken’ subdirectorio y ya no se utiliza. - -Cuando la opción max_parallel_replicas está habilitada, el procesamiento de consultas se paralela en todas las réplicas dentro de un solo fragmento. Para obtener más información, consulte la sección [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). - -## Virtual Columnas {#virtual-columns} - -- `_shard_num` — Contains the `shard_num` (de `system.clusters`). Tipo: [UInt32](../../../sql-reference/data-types/int-uint.md). - -!!! note "Nota" - Ya [`remote`](../../../sql-reference/table-functions/remote.md)/`cluster` funciones de tabla crean internamente instancia temporal del mismo motor distribuido, `_shard_num` está disponible allí también. - -**Ver también** - -- [Virtual columnas](index.md#table_engines-virtual_columns) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/distributed/) diff --git a/docs/es/engines/table-engines/special/external-data.md b/docs/es/engines/table-engines/special/external-data.md deleted file mode 100644 index f2ce4abbb0f..00000000000 --- a/docs/es/engines/table-engines/special/external-data.md +++ /dev/null @@ -1,68 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 34 -toc_title: Datos externos ---- - -# Datos externos para el procesamiento de consultas {#external-data-for-query-processing} - -ClickHouse permite enviar a un servidor los datos necesarios para procesar una consulta, junto con una consulta SELECT. Estos datos se colocan en una tabla temporal (consulte la sección “Temporary tables”) y se puede utilizar en la consulta (por ejemplo, en operadores IN). - -Por ejemplo, si tiene un archivo de texto con identificadores de usuario importantes, puede cargarlo en el servidor junto con una consulta que utilice la filtración de esta lista. - -Si necesita ejecutar más de una consulta con un gran volumen de datos externos, no utilice esta función. Es mejor cargar los datos a la base de datos con anticipación. - -Los datos externos se pueden cargar mediante el cliente de línea de comandos (en modo no interactivo) o mediante la interfaz HTTP. - -En el cliente de línea de comandos, puede especificar una sección de parámetros en el formato - -``` bash ---external --file=... [--name=...] [--format=...] [--types=...|--structure=...] -``` - -Puede tener varias secciones como esta, para el número de tablas que se transmiten. - -**–external** – Marks the beginning of a clause. -**–file** – Path to the file with the table dump, or -, which refers to stdin. -Solo se puede recuperar una sola tabla de stdin. - -Los siguientes parámetros son opcionales: **–name**– Name of the table. If omitted, _data is used. -**–format** – Data format in the file. If omitted, TabSeparated is used. - -Se requiere uno de los siguientes parámetros:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … -**–structure**– The table structure in the format`UserID UInt64`, `URL String`. Define los nombres y tipos de columna. - -Los archivos especificados en ‘file’ se analizará mediante el formato especificado en ‘format’ utilizando los tipos de datos especificados en ‘types’ o ‘structure’. La mesa será cargado en el servidor y accesibles, como una tabla temporal con el nombre de ‘name’. - -Ejemplos: - -``` bash -$ echo -ne "1\n2\n3\n" | clickhouse-client --query="SELECT count() FROM test.visits WHERE TraficSourceID IN _data" --external --file=- --types=Int8 -849897 -$ cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' -/bin/sh 20 -/bin/false 5 -/bin/bash 4 -/usr/sbin/nologin 1 -/bin/sync 1 -``` - -Cuando se utiliza la interfaz HTTP, los datos externos se pasan en el formato multipart/form-data. Cada tabla se transmite como un archivo separado. El nombre de la tabla se toma del nombre del archivo. El ‘query_string’ se pasa los parámetros ‘name_format’, ‘name_types’, y ‘name_structure’, donde ‘name’ es el nombre de la tabla a la que corresponden estos parámetros. El significado de los parámetros es el mismo que cuando se usa el cliente de línea de comandos. - -Ejemplo: - -``` bash -$ cat /etc/passwd | sed 's/:/\t/g' > passwd.tsv - -$ curl -F 'passwd=@passwd.tsv;' 'http://localhost:8123/?query=SELECT+shell,+count()+AS+c+FROM+passwd+GROUP+BY+shell+ORDER+BY+c+DESC&passwd_structure=login+String,+unused+String,+uid+UInt16,+gid+UInt16,+comment+String,+home+String,+shell+String' -/bin/sh 20 -/bin/false 5 -/bin/bash 4 -/usr/sbin/nologin 1 -/bin/sync 1 -``` - -Para el procesamiento de consultas distribuidas, las tablas temporales se envían a todos los servidores remotos. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/external_data/) diff --git a/docs/es/engines/table-engines/special/file.md b/docs/es/engines/table-engines/special/file.md deleted file mode 100644 index fb739506a22..00000000000 --- a/docs/es/engines/table-engines/special/file.md +++ /dev/null @@ -1,90 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: File ---- - -# File {#table_engines-file} - -El motor de tabla de archivos mantiene los datos en un archivo en uno de los [file -formato](../../../interfaces/formats.md#formats) (TabSeparated, Native, etc.). - -Ejemplos de uso: - -- Exportación de datos de ClickHouse a archivo. -- Convertir datos de un formato a otro. -- Actualización de datos en ClickHouse mediante la edición de un archivo en un disco. - -## Uso en el servidor ClickHouse {#usage-in-clickhouse-server} - -``` sql -File(Format) -``` - -El `Format` parámetro especifica uno de los formatos de archivo disponibles. Realizar -`SELECT` consultas, el formato debe ser compatible para la entrada, y para realizar -`INSERT` queries – for output. The available formats are listed in the -[Formato](../../../interfaces/formats.md#formats) apartado. - -ClickHouse no permite especificar la ruta del sistema de archivos para`File`. Utilizará la carpeta definida por [camino](../../../operations/server-configuration-parameters/settings.md) configuración en la configuración del servidor. - -Al crear una tabla usando `File(Format)` crea un subdirectorio vacío en esa carpeta. Cuando los datos se escriben en esa tabla, se colocan en `data.Format` en ese subdirectorio. - -Puede crear manualmente esta subcarpeta y archivo en el sistema de archivos del servidor y luego [ATTACH](../../../sql-reference/statements/misc.md) para mostrar información con el nombre coincidente, para que pueda consultar datos desde ese archivo. - -!!! warning "Advertencia" - Tenga cuidado con esta funcionalidad, ya que ClickHouse no realiza un seguimiento de los cambios externos en dichos archivos. El resultado de las escrituras simultáneas a través de ClickHouse y fuera de ClickHouse no está definido. - -**Ejemplo:** - -**1.** Configurar el `file_engine_table` tabla: - -``` sql -CREATE TABLE file_engine_table (name String, value UInt32) ENGINE=File(TabSeparated) -``` - -Por defecto, ClickHouse creará una carpeta `/var/lib/clickhouse/data/default/file_engine_table`. - -**2.** Crear manualmente `/var/lib/clickhouse/data/default/file_engine_table/data.TabSeparated` contener: - -``` bash -$ cat data.TabSeparated -one 1 -two 2 -``` - -**3.** Consultar los datos: - -``` sql -SELECT * FROM file_engine_table -``` - -``` text -┌─name─┬─value─┐ -│ one │ 1 │ -│ two │ 2 │ -└──────┴───────┘ -``` - -## Uso en ClickHouse-local {#usage-in-clickhouse-local} - -En [Sistema abierto.](../../../operations/utilities/clickhouse-local.md#clickhouse-local) El motor de archivos acepta la ruta del archivo además de `Format`. Los flujos de entrada / salida predeterminados se pueden especificar utilizando nombres numéricos o legibles por humanos como `0` o `stdin`, `1` o `stdout`. -**Ejemplo:** - -``` bash -$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table" -``` - -## Detalles de la implementación {#details-of-implementation} - -- Multiple `SELECT` las consultas se pueden realizar simultáneamente, pero `INSERT` las consultas se esperarán entre sí. -- Apoyado la creación de nuevos archivos por `INSERT` consulta. -- Si el archivo existe, `INSERT` añadiría nuevos valores en él. -- No soportado: - - `ALTER` - - `SELECT ... SAMPLE` - - Indice - - Replicación - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/file/) diff --git a/docs/es/engines/table-engines/special/generate.md b/docs/es/engines/table-engines/special/generate.md deleted file mode 100644 index 67e664284b4..00000000000 --- a/docs/es/engines/table-engines/special/generate.md +++ /dev/null @@ -1,61 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 46 -toc_title: GenerateRandom ---- - -# Generaterandom {#table_engines-generate} - -El motor de tabla GenerateRandom produce datos aleatorios para el esquema de tabla determinado. - -Ejemplos de uso: - -- Se usa en la prueba para poblar una tabla grande reproducible. -- Generar entrada aleatoria para pruebas de fuzzing. - -## Uso en el servidor ClickHouse {#usage-in-clickhouse-server} - -``` sql -ENGINE = GenerateRandom(random_seed, max_string_length, max_array_length) -``` - -El `max_array_length` y `max_string_length` parámetros especifican la longitud máxima de todos -columnas y cadenas de matriz correspondientemente en los datos generados. - -Generar motor de tabla sólo admite `SELECT` consulta. - -Es compatible con todos [Tipos de datos](../../../sql-reference/data-types/index.md) que se pueden almacenar en una tabla excepto `LowCardinality` y `AggregateFunction`. - -**Ejemplo:** - -**1.** Configurar el `generate_engine_table` tabla: - -``` sql -CREATE TABLE generate_engine_table (name String, value UInt32) ENGINE = GenerateRandom(1, 5, 3) -``` - -**2.** Consultar los datos: - -``` sql -SELECT * FROM generate_engine_table LIMIT 3 -``` - -``` text -┌─name─┬──────value─┐ -│ c4xJ │ 1412771199 │ -│ r │ 1791099446 │ -│ 7#$ │ 124312908 │ -└──────┴────────────┘ -``` - -## Detalles de la implementación {#details-of-implementation} - -- No soportado: - - `ALTER` - - `SELECT ... SAMPLE` - - `INSERT` - - Indice - - Replicación - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/generate/) diff --git a/docs/es/engines/table-engines/special/index.md b/docs/es/engines/table-engines/special/index.md deleted file mode 100644 index 9927a1f61d9..00000000000 --- a/docs/es/engines/table-engines/special/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Especial -toc_priority: 31 ---- - - diff --git a/docs/es/engines/table-engines/special/join.md b/docs/es/engines/table-engines/special/join.md deleted file mode 100644 index 83e21b7c8cc..00000000000 --- a/docs/es/engines/table-engines/special/join.md +++ /dev/null @@ -1,111 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: Unir ---- - -# Unir {#join} - -Estructura de datos preparada para usar en [JOIN](../../../sql-reference/statements/select/join.md#select-join) operación. - -## Creación de una tabla {#creating-a-table} - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], -) ENGINE = Join(join_strictness, join_type, k1[, k2, ...]) -``` - -Vea la descripción detallada del [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) consulta. - -**Parámetros del motor** - -- `join_strictness` – [ÚNETE a la rigurosidad](../../../sql-reference/statements/select/join.md#select-join-types). -- `join_type` – [Tipo de unión](../../../sql-reference/statements/select/join.md#select-join-types). -- `k1[, k2, ...]` – Key columns from the `USING` cláusula que el `JOIN` operación se hace con. - -Entrar `join_strictness` y `join_type` parámetros sin comillas, por ejemplo, `Join(ANY, LEFT, col1)`. Deben coincidir con el `JOIN` operación para la que se utilizará la tabla. Si los parámetros no coinciden, ClickHouse no lanza una excepción y puede devolver datos incorrectos. - -## Uso de la tabla {#table-usage} - -### Ejemplo {#example} - -Creación de la tabla del lado izquierdo: - -``` sql -CREATE TABLE id_val(`id` UInt32, `val` UInt32) ENGINE = TinyLog -``` - -``` sql -INSERT INTO id_val VALUES (1,11)(2,12)(3,13) -``` - -Creando el lado derecho `Join` tabla: - -``` sql -CREATE TABLE id_val_join(`id` UInt32, `val` UInt8) ENGINE = Join(ANY, LEFT, id) -``` - -``` sql -INSERT INTO id_val_join VALUES (1,21)(1,22)(3,23) -``` - -Unirse a las tablas: - -``` sql -SELECT * FROM id_val ANY LEFT JOIN id_val_join USING (id) SETTINGS join_use_nulls = 1 -``` - -``` text -┌─id─┬─val─┬─id_val_join.val─┐ -│ 1 │ 11 │ 21 │ -│ 2 │ 12 │ ᴺᵁᴸᴸ │ -│ 3 │ 13 │ 23 │ -└────┴─────┴─────────────────┘ -``` - -Como alternativa, puede recuperar datos del `Join` tabla, especificando el valor de la clave de unión: - -``` sql -SELECT joinGet('id_val_join', 'val', toUInt32(1)) -``` - -``` text -┌─joinGet('id_val_join', 'val', toUInt32(1))─┐ -│ 21 │ -└────────────────────────────────────────────┘ -``` - -### Selección e inserción de datos {#selecting-and-inserting-data} - -Usted puede utilizar `INSERT` consultas para agregar datos al `Join`-mesas de motor. Si la tabla se creó con el `ANY` estricta, se ignoran los datos de las claves duplicadas. Con el `ALL` estricta, se agregan todas las filas. - -No se puede realizar una `SELECT` consulta directamente desde la tabla. En su lugar, use uno de los siguientes métodos: - -- Coloque la mesa hacia el lado derecho en un `JOIN` clausula. -- Llame al [joinGet](../../../sql-reference/functions/other-functions.md#joinget) función, que le permite extraer datos de la tabla de la misma manera que de un diccionario. - -### Limitaciones y ajustes {#join-limitations-and-settings} - -Al crear una tabla, se aplican los siguientes valores: - -- [Sistema abierto.](../../../operations/settings/settings.md#join_use_nulls) -- [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) -- [Método de codificación de datos:](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) -- [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) -- [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) - -El `Join`-las tablas del motor no se pueden usar en `GLOBAL JOIN` operación. - -El `Join`-motor permite el uso [Sistema abierto.](../../../operations/settings/settings.md#join_use_nulls) ajuste en el `CREATE TABLE` instrucción. Y [SELECT](../../../sql-reference/statements/select/index.md) consulta permite el uso `join_use_nulls` demasiado. Si tienes diferentes `join_use_nulls` configuración, puede obtener un error al unirse a la tabla. Depende del tipo de JOIN. Cuando se utiliza [joinGet](../../../sql-reference/functions/other-functions.md#joinget) función, usted tiene que utilizar el mismo `join_use_nulls` ajuste en `CRATE TABLE` y `SELECT` instrucción. - -## Almacenamiento de datos {#data-storage} - -`Join` datos de la tabla siempre se encuentra en la memoria RAM. Al insertar filas en una tabla, ClickHouse escribe bloques de datos en el directorio del disco para que puedan restaurarse cuando se reinicie el servidor. - -Si el servidor se reinicia incorrectamente, el bloque de datos en el disco puede perderse o dañarse. En este caso, es posible que deba eliminar manualmente el archivo con datos dañados. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/join/) diff --git a/docs/es/engines/table-engines/special/materializedview.md b/docs/es/engines/table-engines/special/materializedview.md deleted file mode 100644 index 87e5218eb6a..00000000000 --- a/docs/es/engines/table-engines/special/materializedview.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 43 -toc_title: "M\xE9todo de codificaci\xF3n de datos:" ---- - -# Método de codificación de datos: {#materializedview} - -Se utiliza para implementar vistas materializadas (para obtener más información, consulte [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query)). Para almacenar datos, utiliza un motor diferente que se especificó al crear la vista. Al leer desde una tabla, solo usa este motor. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/materializedview/) diff --git a/docs/es/engines/table-engines/special/memory.md b/docs/es/engines/table-engines/special/memory.md deleted file mode 100644 index 3d4f8ddff54..00000000000 --- a/docs/es/engines/table-engines/special/memory.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: Memoria ---- - -# Memoria {#memory} - -El motor de memoria almacena datos en RAM, en forma sin comprimir. Los datos se almacenan exactamente en la misma forma en que se reciben cuando se leen. En otras palabras, la lectura de esta tabla es completamente gratuita. -El acceso a los datos simultáneos está sincronizado. Los bloqueos son cortos: las operaciones de lectura y escritura no se bloquean entre sí. -Los índices no son compatibles. La lectura está paralelizada. -La productividad máxima (más de 10 GB/s) se alcanza en consultas simples, porque no hay lectura del disco, descomprimir o deserializar datos. (Cabe señalar que, en muchos casos, la productividad del motor MergeTree es casi tan alta.) -Al reiniciar un servidor, los datos desaparecen de la tabla y la tabla queda vacía. -Normalmente, el uso de este motor de tabla no está justificado. Sin embargo, se puede usar para pruebas y para tareas donde se requiere la velocidad máxima en un número relativamente pequeño de filas (hasta aproximadamente 100,000,000). - -El sistema utiliza el motor de memoria para tablas temporales con datos de consulta externos (consulte la sección “External data for processing a query”), y para la implementación de GLOBAL IN (véase la sección “IN operators”). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/memory/) diff --git a/docs/es/engines/table-engines/special/merge.md b/docs/es/engines/table-engines/special/merge.md deleted file mode 100644 index 6ed2c272914..00000000000 --- a/docs/es/engines/table-engines/special/merge.md +++ /dev/null @@ -1,70 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: Fusionar ---- - -# Fusionar {#merge} - -El `Merge` motor (no debe confundirse con `MergeTree`) no almacena datos en sí, pero permite leer de cualquier número de otras tablas simultáneamente. -La lectura se paralela automáticamente. No se admite la escritura en una tabla. Al leer, se usan los índices de las tablas que realmente se están leyendo, si existen. -El `Merge` engine acepta parámetros: el nombre de la base de datos y una expresión regular para las tablas. - -Ejemplo: - -``` sql -Merge(hits, '^WatchLog') -``` - -Los datos se leerán de las tablas en el `hits` base de datos que tienen nombres que coinciden con la expresión regular ‘`^WatchLog`’. - -En lugar del nombre de la base de datos, puede usar una expresión constante que devuelva una cadena. Por ejemplo, `currentDatabase()`. - -Regular expressions — [Re2](https://github.com/google/re2) (soporta un subconjunto de PCRE), sensible a mayúsculas y minúsculas. -Vea las notas sobre los símbolos de escape en expresiones regulares en el “match” apartado. - -Al seleccionar tablas para leer, el `Merge` no se seleccionará la tabla en sí, incluso si coincide con la expresión regular. Esto es para evitar bucles. -Es posible crear dos `Merge` tablas que intentarán interminablemente leer los datos de los demás, pero esta no es una buena idea. - -La forma típica de usar el `Merge` para trabajar con un gran número de `TinyLog` tablas como si con una sola tabla. - -Ejemplo 2: - -Digamos que tiene una tabla antigua (WatchLog_old) y decidió cambiar la partición sin mover datos a una nueva tabla (WatchLog_new) y necesita ver datos de ambas tablas. - -``` sql -CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) -ENGINE=MergeTree(date, (UserId, EventType), 8192); -INSERT INTO WatchLog_old VALUES ('2018-01-01', 1, 'hit', 3); - -CREATE TABLE WatchLog_new(date Date, UserId Int64, EventType String, Cnt UInt64) -ENGINE=MergeTree PARTITION BY date ORDER BY (UserId, EventType) SETTINGS index_granularity=8192; -INSERT INTO WatchLog_new VALUES ('2018-01-02', 2, 'hit', 3); - -CREATE TABLE WatchLog as WatchLog_old ENGINE=Merge(currentDatabase(), '^WatchLog'); - -SELECT * -FROM WatchLog -``` - -``` text -┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ -│ 2018-01-01 │ 1 │ hit │ 3 │ -└────────────┴────────┴───────────┴─────┘ -┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ -│ 2018-01-02 │ 2 │ hit │ 3 │ -└────────────┴────────┴───────────┴─────┘ -``` - -## Virtual Columnas {#virtual-columns} - -- `_table` — Contains the name of the table from which data was read. Type: [Cadena](../../../sql-reference/data-types/string.md). - - Puede establecer las condiciones constantes en `_table` en el `WHERE/PREWHERE` cláusula (por ejemplo, `WHERE _table='xyz'`). En este caso, la operación de lectura se realiza sólo para las tablas donde la condición en `_table` está satisfecho, por lo que el `_table` columna actúa como un índice. - -**Ver también** - -- [Virtual columnas](index.md#table_engines-virtual_columns) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/merge/) diff --git a/docs/es/engines/table-engines/special/null.md b/docs/es/engines/table-engines/special/null.md deleted file mode 100644 index cc05e7839c9..00000000000 --- a/docs/es/engines/table-engines/special/null.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: Nulo ---- - -# Nulo {#null} - -Al escribir en una tabla Null, los datos se ignoran. Al leer desde una tabla Null, la respuesta está vacía. - -Sin embargo, puede crear una vista materializada en una tabla Null. Entonces los datos escritos en la tabla terminarán en la vista. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/null/) diff --git a/docs/es/engines/table-engines/special/set.md b/docs/es/engines/table-engines/special/set.md deleted file mode 100644 index 4ff23202443..00000000000 --- a/docs/es/engines/table-engines/special/set.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: Establecer ---- - -# Establecer {#set} - -Un conjunto de datos que siempre está en la memoria RAM. Está diseñado para su uso en el lado derecho del operador IN (consulte la sección “IN operators”). - -Puede usar INSERT para insertar datos en la tabla. Se agregarán nuevos elementos al conjunto de datos, mientras que los duplicados se ignorarán. -Pero no puede realizar SELECT desde la tabla. La única forma de recuperar datos es usándolos en la mitad derecha del operador IN. - -Los datos siempre se encuentran en la memoria RAM. Para INSERT, los bloques de datos insertados también se escriben en el directorio de tablas en el disco. Al iniciar el servidor, estos datos se cargan en la RAM. En otras palabras, después de reiniciar, los datos permanecen en su lugar. - -Para un reinicio aproximado del servidor, el bloque de datos en el disco puede perderse o dañarse. En este último caso, es posible que deba eliminar manualmente el archivo con datos dañados. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/set/) diff --git a/docs/es/engines/table-engines/special/url.md b/docs/es/engines/table-engines/special/url.md deleted file mode 100644 index 654b8e99a4e..00000000000 --- a/docs/es/engines/table-engines/special/url.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: URL ---- - -# URL(URL, Formato) {#table_engines-url} - -Administra datos en un servidor HTTP/HTTPS remoto. Este motor es similar -a la [File](file.md) motor. - -## Uso del motor en el servidor ClickHouse {#using-the-engine-in-the-clickhouse-server} - -El `format` debe ser uno que ClickHouse pueda usar en -`SELECT` consultas y, si es necesario, en `INSERTs`. Para obtener la lista completa de formatos admitidos, consulte -[Formato](../../../interfaces/formats.md#formats). - -El `URL` debe ajustarse a la estructura de un localizador uniforme de recursos. La dirección URL especificada debe apuntar a un servidor -que utiliza HTTP o HTTPS. Esto no requiere ningún -encabezados adicionales para obtener una respuesta del servidor. - -`INSERT` y `SELECT` las consultas se transforman en `POST` y `GET` peticiones, -respectivamente. Para el procesamiento `POST` solicitudes, el servidor remoto debe admitir -[Codificación de transferencia fragmentada](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). - -Puede limitar el número máximo de saltos de redirección HTTP GET utilizando el [Nombre de la red inalámbrica (SSID):](../../../operations/settings/settings.md#setting-max_http_get_redirects) configuración. - -**Ejemplo:** - -**1.** Crear un `url_engine_table` tabla en el servidor : - -``` sql -CREATE TABLE url_engine_table (word String, value UInt64) -ENGINE=URL('http://127.0.0.1:12345/', CSV) -``` - -**2.** Cree un servidor HTTP básico utilizando las herramientas estándar de Python 3 y -comenzarlo: - -``` python3 -from http.server import BaseHTTPRequestHandler, HTTPServer - -class CSVHTTPServer(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(200) - self.send_header('Content-type', 'text/csv') - self.end_headers() - - self.wfile.write(bytes('Hello,1\nWorld,2\n', "utf-8")) - -if __name__ == "__main__": - server_address = ('127.0.0.1', 12345) - HTTPServer(server_address, CSVHTTPServer).serve_forever() -``` - -``` bash -$ python3 server.py -``` - -**3.** Solicitar datos: - -``` sql -SELECT * FROM url_engine_table -``` - -``` text -┌─word──┬─value─┐ -│ Hello │ 1 │ -│ World │ 2 │ -└───────┴───────┘ -``` - -## Detalles de la implementación {#details-of-implementation} - -- Las lecturas y escrituras pueden ser paralelas -- No soportado: - - `ALTER` y `SELECT...SAMPLE` operación. - - Índices. - - Replicación. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/url/) diff --git a/docs/es/engines/table-engines/special/view.md b/docs/es/engines/table-engines/special/view.md deleted file mode 100644 index dbb496bcca4..00000000000 --- a/docs/es/engines/table-engines/special/view.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 42 -toc_title: Vista ---- - -# Vista {#table_engines-view} - -Se utiliza para implementar vistas (para obtener más información, consulte `CREATE VIEW query`). No almacena datos, pero solo almacena los datos especificados `SELECT` consulta. Al leer desde una tabla, ejecuta esta consulta (y elimina todas las columnas innecesarias de la consulta). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/view/) diff --git a/docs/es/faq/general.md b/docs/es/faq/general.md deleted file mode 100644 index f8446e99152..00000000000 --- a/docs/es/faq/general.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 78 -toc_title: Preguntas generales ---- - -# Preguntas generales {#general-questions} - -## ¿Por qué no usar algo como MapReduce? {#why-not-use-something-like-mapreduce} - -Podemos referirnos a sistemas como MapReduce como sistemas informáticos distribuidos en los que la operación de reducción se basa en la clasificación distribuida. La solución de código abierto más común en esta clase es [Acerca de nosotros](http://hadoop.apache.org). Yandex utiliza su solución interna, YT. - -Estos sistemas no son apropiados para consultas en línea debido a su alta latencia. En otras palabras, no se pueden usar como back-end para una interfaz web. Estos tipos de sistemas no son útiles para actualizaciones de datos en tiempo real. La clasificación distribuida no es la mejor manera de realizar operaciones de reducción si el resultado de la operación y todos los resultados intermedios (si los hay) se encuentran en la RAM de un único servidor, que generalmente es el caso de las consultas en línea. En tal caso, una tabla hash es una forma óptima de realizar operaciones de reducción. Un enfoque común para optimizar las tareas de reducción de mapas es la preagregación (reducción parcial) utilizando una tabla hash en RAM. El usuario realiza esta optimización manualmente. La clasificación distribuida es una de las principales causas de un rendimiento reducido cuando se ejecutan tareas simples de reducción de mapas. - -La mayoría de las implementaciones de MapReduce le permiten ejecutar código arbitrario en un clúster. Pero un lenguaje de consulta declarativo es más adecuado para OLAP para ejecutar experimentos rápidamente. Por ejemplo, Hadoop tiene Hive y Pig. También considere Cloudera Impala o Shark (obsoleto) para Spark, así como Spark SQL, Presto y Apache Drill. El rendimiento cuando se ejecutan tales tareas es muy subóptimo en comparación con los sistemas especializados, pero la latencia relativamente alta hace que sea poco realista utilizar estos sistemas como back-end para una interfaz web. - -## ¿Qué sucede si tengo un problema con las codificaciones al usar Oracle a través de ODBC? {#oracle-odbc-encodings} - -Si utiliza Oracle a través del controlador ODBC como fuente de diccionarios externos, debe establecer el valor `NLS_LANG` variable de entorno en `/etc/default/clickhouse`. Para obtener más información, consulte [Oracle NLS_LANG Preguntas frecuentes](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html). - -**Ejemplo** - -``` sql -NLS_LANG=RUSSIAN_RUSSIA.UTF8 -``` - -## Cómo exporto datos de ClickHouse a un archivo? {#how-to-export-to-file} - -### Uso de la cláusula INTO OUTFILE {#using-into-outfile-clause} - -Añadir un [INTO OUTFILE](../sql-reference/statements/select/into-outfile.md#into-outfile-clause) cláusula a su consulta. - -Por ejemplo: - -``` sql -SELECT * FROM table INTO OUTFILE 'file' -``` - -De forma predeterminada, ClickHouse usa el [TabSeparated](../interfaces/formats.md#tabseparated) formato de datos de salida. Para seleccionar el [formato de datos](../interfaces/formats.md), utilizar el [Cláusula FORMAT](../sql-reference/statements/select/format.md#format-clause). - -Por ejemplo: - -``` sql -SELECT * FROM table INTO OUTFILE 'file' FORMAT CSV -``` - -### Uso de una tabla de motor de archivo {#using-a-file-engine-table} - -Ver [File](../engines/table-engines/special/file.md). - -### Uso de la redirección de línea de comandos {#using-command-line-redirection} - -``` sql -$ clickhouse-client --query "SELECT * from table" --format FormatName > result.txt -``` - -Ver [Casa de clics-cliente](../interfaces/cli.md). - -{## [Artículo Original](https://clickhouse.tech/docs/en/faq/general/) ##} diff --git a/docs/es/faq/index.md b/docs/es/faq/index.md deleted file mode 100644 index a44dbb31e89..00000000000 --- a/docs/es/faq/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: F.A.Q. -toc_priority: 76 ---- - - diff --git a/docs/es/getting-started/example-datasets/amplab-benchmark.md b/docs/es/getting-started/example-datasets/amplab-benchmark.md deleted file mode 100644 index 066bf036266..00000000000 --- a/docs/es/getting-started/example-datasets/amplab-benchmark.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 17 -toc_title: Referencia de Big Data de AMPLab ---- - -# Referencia de Big Data de AMPLab {#amplab-big-data-benchmark} - -Ver https://amplab.cs.berkeley.edu/benchmark/ - -Regístrese para obtener una cuenta gratuita en https://aws.amazon.com. Requiere una tarjeta de crédito, correo electrónico y número de teléfono. Obtenga una nueva clave de acceso en https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential - -Ejecute lo siguiente en la consola: - -``` bash -$ sudo apt-get install s3cmd -$ mkdir tiny; cd tiny; -$ s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/tiny/ . -$ cd .. -$ mkdir 1node; cd 1node; -$ s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/1node/ . -$ cd .. -$ mkdir 5nodes; cd 5nodes; -$ s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/5nodes/ . -$ cd .. -``` - -Ejecute las siguientes consultas de ClickHouse: - -``` sql -CREATE TABLE rankings_tiny -( - pageURL String, - pageRank UInt32, - avgDuration UInt32 -) ENGINE = Log; - -CREATE TABLE uservisits_tiny -( - sourceIP String, - destinationURL String, - visitDate Date, - adRevenue Float32, - UserAgent String, - cCode FixedString(3), - lCode FixedString(6), - searchWord String, - duration UInt32 -) ENGINE = MergeTree(visitDate, visitDate, 8192); - -CREATE TABLE rankings_1node -( - pageURL String, - pageRank UInt32, - avgDuration UInt32 -) ENGINE = Log; - -CREATE TABLE uservisits_1node -( - sourceIP String, - destinationURL String, - visitDate Date, - adRevenue Float32, - UserAgent String, - cCode FixedString(3), - lCode FixedString(6), - searchWord String, - duration UInt32 -) ENGINE = MergeTree(visitDate, visitDate, 8192); - -CREATE TABLE rankings_5nodes_on_single -( - pageURL String, - pageRank UInt32, - avgDuration UInt32 -) ENGINE = Log; - -CREATE TABLE uservisits_5nodes_on_single -( - sourceIP String, - destinationURL String, - visitDate Date, - adRevenue Float32, - UserAgent String, - cCode FixedString(3), - lCode FixedString(6), - searchWord String, - duration UInt32 -) ENGINE = MergeTree(visitDate, visitDate, 8192); -``` - -Volver a la consola: - -``` bash -$ for i in tiny/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_tiny FORMAT CSV"; done -$ for i in tiny/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_tiny FORMAT CSV"; done -$ for i in 1node/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_1node FORMAT CSV"; done -$ for i in 1node/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_1node FORMAT CSV"; done -$ for i in 5nodes/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_5nodes_on_single FORMAT CSV"; done -$ for i in 5nodes/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_5nodes_on_single FORMAT CSV"; done -``` - -Consultas para obtener muestras de datos: - -``` sql -SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000 - -SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8) - -SELECT - sourceIP, - sum(adRevenue) AS totalRevenue, - avg(pageRank) AS pageRank -FROM rankings_1node ALL INNER JOIN -( - SELECT - sourceIP, - destinationURL AS pageURL, - adRevenue - FROM uservisits_1node - WHERE (visitDate > '1980-01-01') AND (visitDate < '1980-04-01') -) USING pageURL -GROUP BY sourceIP -ORDER BY totalRevenue DESC -LIMIT 1 -``` - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/amplab_benchmark/) diff --git a/docs/es/getting-started/example-datasets/criteo.md b/docs/es/getting-started/example-datasets/criteo.md deleted file mode 100644 index 79203b0276d..00000000000 --- a/docs/es/getting-started/example-datasets/criteo.md +++ /dev/null @@ -1,81 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 19 -toc_title: Registros de clics de Terabyte de Criteo ---- - -# Terabyte de registros de clics de Criteo {#terabyte-of-click-logs-from-criteo} - -Descargue los datos de http://labs.criteo.com/downloads/download-terabyte-click-logs/ - -Cree una tabla para importar el registro: - -``` sql -CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log -``` - -Descargar los datos: - -``` bash -$ for i in {00..23}; do echo $i; zcat datasets/criteo/day_${i#0}.gz | sed -r 's/^/2000-01-'${i/00/24}'\t/' | clickhouse-client --host=example-perftest01j --query="INSERT INTO criteo_log FORMAT TabSeparated"; done -``` - -Crear una tabla para los datos convertidos: - -``` sql -CREATE TABLE criteo -( - date Date, - clicked UInt8, - int1 Int32, - int2 Int32, - int3 Int32, - int4 Int32, - int5 Int32, - int6 Int32, - int7 Int32, - int8 Int32, - int9 Int32, - int10 Int32, - int11 Int32, - int12 Int32, - int13 Int32, - icat1 UInt32, - icat2 UInt32, - icat3 UInt32, - icat4 UInt32, - icat5 UInt32, - icat6 UInt32, - icat7 UInt32, - icat8 UInt32, - icat9 UInt32, - icat10 UInt32, - icat11 UInt32, - icat12 UInt32, - icat13 UInt32, - icat14 UInt32, - icat15 UInt32, - icat16 UInt32, - icat17 UInt32, - icat18 UInt32, - icat19 UInt32, - icat20 UInt32, - icat21 UInt32, - icat22 UInt32, - icat23 UInt32, - icat24 UInt32, - icat25 UInt32, - icat26 UInt32 -) ENGINE = MergeTree(date, intHash32(icat1), (date, intHash32(icat1)), 8192) -``` - -Transforme los datos del registro sin procesar y colóquelos en la segunda tabla: - -``` sql -INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; - -DROP TABLE criteo_log; -``` - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/criteo/) diff --git a/docs/es/getting-started/example-datasets/index.md b/docs/es/getting-started/example-datasets/index.md deleted file mode 100644 index 28e06987af1..00000000000 --- a/docs/es/getting-started/example-datasets/index.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Datos De Ejemplo -toc_priority: 12 -toc_title: "Implantaci\xF3n" ---- - -# Datos De Ejemplo {#example-datasets} - -En esta sección se describe cómo obtener conjuntos de datos de ejemplo e importarlos a ClickHouse. -Para algunos conjuntos de datos también están disponibles consultas de ejemplo. - -- [Yandex anonimizado.Conjunto de datos de Metrica](metrica.md) -- [Estrella Schema Benchmark](star-schema.md) -- [Nombre de la red inalámbrica (SSID):](wikistat.md) -- [Terabyte de registros de clics de Criteo](criteo.md) -- [Referencia de Big Data de AMPLab](amplab-benchmark.md) -- [Datos de taxis de Nueva York](nyc-taxi.md) -- [A tiempo](ontime.md) - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets) diff --git a/docs/es/getting-started/example-datasets/metrica.md b/docs/es/getting-started/example-datasets/metrica.md deleted file mode 100644 index 0b3bc8b6833..00000000000 --- a/docs/es/getting-started/example-datasets/metrica.md +++ /dev/null @@ -1,70 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 14 -toc_title: El Yandex.Metrica Datos ---- - -# Yandex anonimizado.Metrica Datos {#anonymized-yandex-metrica-data} - -El conjunto de datos consta de dos tablas que contienen datos anónimos sobre los hits (`hits_v1`) y visitas (`visits_v1`) el Yandex.Métrica. Puedes leer más sobre Yandex.Metrica en [Historial de ClickHouse](../../introduction/history.md) apartado. - -El conjunto de datos consta de dos tablas, cualquiera de ellas se puede descargar como `tsv.xz` o como particiones preparadas. Además, una versión extendida de la `hits` La tabla que contiene 100 millones de filas está disponible como TSV en https://datasets.clickhouse.tech/hits/tsv/hits_100m_obfuscated_v1.tsv.xz y como particiones preparadas en https://datasets.clickhouse.tech/hits/partitions/hits_100m_obfuscated_v1.tar.xz. - -## Obtención de tablas a partir de particiones preparadas {#obtaining-tables-from-prepared-partitions} - -Descargar e importar tabla de hits: - -``` bash -curl -O https://datasets.clickhouse.tech/hits/partitions/hits_v1.tar -tar xvf hits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory -# check permissions on unpacked data, fix if required -sudo service clickhouse-server restart -clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" -``` - -Descargar e importar visitas: - -``` bash -curl -O https://datasets.clickhouse.tech/visits/partitions/visits_v1.tar -tar xvf visits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory -# check permissions on unpacked data, fix if required -sudo service clickhouse-server restart -clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" -``` - -## Obtención de tablas a partir de un archivo TSV comprimido {#obtaining-tables-from-compressed-tsv-file} - -Descargar e importar hits desde un archivo TSV comprimido: - -``` bash -curl https://datasets.clickhouse.tech/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv -# now create table -clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -# import data -cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 -# optionally you can optimize table -clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" -clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" -``` - -Descargue e importe visitas desde un archivo tsv comprimido: - -``` bash -curl https://datasets.clickhouse.tech/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv -# now create table -clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -# import data -cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 -# optionally you can optimize table -clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" -clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" -``` - -## Consultas de ejemplo {#example-queries} - -[Tutorial de ClickHouse](../../getting-started/tutorial.md) se basa en Yandex.El conjunto de datos de Metrica y la forma recomendada de comenzar con este conjunto de datos es simplemente pasar por el tutorial. - -Se pueden encontrar ejemplos adicionales de consultas a estas tablas entre [pruebas estatales](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/1_stateful) de ClickHouse (se nombran `test.hists` y `test.visits` alli). diff --git a/docs/es/getting-started/example-datasets/nyc-taxi.md b/docs/es/getting-started/example-datasets/nyc-taxi.md deleted file mode 100644 index c6441311c96..00000000000 --- a/docs/es/getting-started/example-datasets/nyc-taxi.md +++ /dev/null @@ -1,390 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 16 -toc_title: Datos de taxis de Nueva York ---- - -# Datos de taxis de Nueva York {#new-york-taxi-data} - -Este conjunto de datos se puede obtener de dos maneras: - -- importación de datos sin procesar -- descarga de particiones preparadas - -## Cómo importar los datos sin procesar {#how-to-import-the-raw-data} - -Consulte https://github.com/toddwschneider/nyc-taxi-data y http://tech.marksblogg.com/billion-nyc-taxi-rides-redshift.html para obtener la descripción de un conjunto de datos e instrucciones para descargar. - -La descarga dará como resultado aproximadamente 227 GB de datos sin comprimir en archivos CSV. La descarga tarda aproximadamente una hora en una conexión de 1 Gbit (la descarga paralela de s3.amazonaws.com recupera al menos la mitad de un canal de 1 Gbit). -Es posible que algunos de los archivos no se descarguen por completo. Verifique los tamaños de archivo y vuelva a descargar cualquiera que parezca dudoso. - -Algunos de los archivos pueden contener filas no válidas. Puede arreglarlos de la siguiente manera: - -``` bash -sed -E '/(.*,){18,}/d' data/yellow_tripdata_2010-02.csv > data/yellow_tripdata_2010-02.csv_ -sed -E '/(.*,){18,}/d' data/yellow_tripdata_2010-03.csv > data/yellow_tripdata_2010-03.csv_ -mv data/yellow_tripdata_2010-02.csv_ data/yellow_tripdata_2010-02.csv -mv data/yellow_tripdata_2010-03.csv_ data/yellow_tripdata_2010-03.csv -``` - -Entonces los datos deben ser preprocesados en PostgreSQL. Esto creará selecciones de puntos en los polígonos (para hacer coincidir los puntos en el mapa con los distritos de la ciudad de Nueva York) y combinará todos los datos en una única tabla plana desnormalizada mediante el uso de una unión. Para hacer esto, deberá instalar PostgreSQL con soporte PostGIS. - -Tenga cuidado al correr `initialize_database.sh` y volver a verificar manualmente que todas las tablas se crearon correctamente. - -Se tarda entre 20 y 30 minutos en procesar los datos de cada mes en PostgreSQL, por un total de aproximadamente 48 horas. - -Puede comprobar el número de filas descargadas de la siguiente manera: - -``` bash -$ time psql nyc-taxi-data -c "SELECT count(*) FROM trips;" -## Count - 1298979494 -(1 row) - -real 7m9.164s -``` - -(Esto es un poco más de 1.1 mil millones de filas reportadas por Mark Litwintschik en una serie de publicaciones de blog.) - -Los datos en PostgreSQL utilizan 370 GB de espacio. - -Exportación de los datos de PostgreSQL: - -``` sql -COPY -( - SELECT trips.id, - trips.vendor_id, - trips.pickup_datetime, - trips.dropoff_datetime, - trips.store_and_fwd_flag, - trips.rate_code_id, - trips.pickup_longitude, - trips.pickup_latitude, - trips.dropoff_longitude, - trips.dropoff_latitude, - trips.passenger_count, - trips.trip_distance, - trips.fare_amount, - trips.extra, - trips.mta_tax, - trips.tip_amount, - trips.tolls_amount, - trips.ehail_fee, - trips.improvement_surcharge, - trips.total_amount, - trips.payment_type, - trips.trip_type, - trips.pickup, - trips.dropoff, - - cab_types.type cab_type, - - weather.precipitation_tenths_of_mm rain, - weather.snow_depth_mm, - weather.snowfall_mm, - weather.max_temperature_tenths_degrees_celsius max_temp, - weather.min_temperature_tenths_degrees_celsius min_temp, - weather.average_wind_speed_tenths_of_meters_per_second wind, - - pick_up.gid pickup_nyct2010_gid, - pick_up.ctlabel pickup_ctlabel, - pick_up.borocode pickup_borocode, - pick_up.boroname pickup_boroname, - pick_up.ct2010 pickup_ct2010, - pick_up.boroct2010 pickup_boroct2010, - pick_up.cdeligibil pickup_cdeligibil, - pick_up.ntacode pickup_ntacode, - pick_up.ntaname pickup_ntaname, - pick_up.puma pickup_puma, - - drop_off.gid dropoff_nyct2010_gid, - drop_off.ctlabel dropoff_ctlabel, - drop_off.borocode dropoff_borocode, - drop_off.boroname dropoff_boroname, - drop_off.ct2010 dropoff_ct2010, - drop_off.boroct2010 dropoff_boroct2010, - drop_off.cdeligibil dropoff_cdeligibil, - drop_off.ntacode dropoff_ntacode, - drop_off.ntaname dropoff_ntaname, - drop_off.puma dropoff_puma - FROM trips - LEFT JOIN cab_types - ON trips.cab_type_id = cab_types.id - LEFT JOIN central_park_weather_observations_raw weather - ON weather.date = trips.pickup_datetime::date - LEFT JOIN nyct2010 pick_up - ON pick_up.gid = trips.pickup_nyct2010_gid - LEFT JOIN nyct2010 drop_off - ON drop_off.gid = trips.dropoff_nyct2010_gid -) TO '/opt/milovidov/nyc-taxi-data/trips.tsv'; -``` - -La instantánea de datos se crea a una velocidad de aproximadamente 50 MB por segundo. Al crear la instantánea, PostgreSQL lee desde el disco a una velocidad de aproximadamente 28 MB por segundo. -Esto toma alrededor de 5 horas. El archivo TSV resultante es 590612904969 bytes. - -Crear una tabla temporal en ClickHouse: - -``` sql -CREATE TABLE trips -( -trip_id UInt32, -vendor_id String, -pickup_datetime DateTime, -dropoff_datetime Nullable(DateTime), -store_and_fwd_flag Nullable(FixedString(1)), -rate_code_id Nullable(UInt8), -pickup_longitude Nullable(Float64), -pickup_latitude Nullable(Float64), -dropoff_longitude Nullable(Float64), -dropoff_latitude Nullable(Float64), -passenger_count Nullable(UInt8), -trip_distance Nullable(Float64), -fare_amount Nullable(Float32), -extra Nullable(Float32), -mta_tax Nullable(Float32), -tip_amount Nullable(Float32), -tolls_amount Nullable(Float32), -ehail_fee Nullable(Float32), -improvement_surcharge Nullable(Float32), -total_amount Nullable(Float32), -payment_type Nullable(String), -trip_type Nullable(UInt8), -pickup Nullable(String), -dropoff Nullable(String), -cab_type Nullable(String), -precipitation Nullable(UInt8), -snow_depth Nullable(UInt8), -snowfall Nullable(UInt8), -max_temperature Nullable(UInt8), -min_temperature Nullable(UInt8), -average_wind_speed Nullable(UInt8), -pickup_nyct2010_gid Nullable(UInt8), -pickup_ctlabel Nullable(String), -pickup_borocode Nullable(UInt8), -pickup_boroname Nullable(String), -pickup_ct2010 Nullable(String), -pickup_boroct2010 Nullable(String), -pickup_cdeligibil Nullable(FixedString(1)), -pickup_ntacode Nullable(String), -pickup_ntaname Nullable(String), -pickup_puma Nullable(String), -dropoff_nyct2010_gid Nullable(UInt8), -dropoff_ctlabel Nullable(String), -dropoff_borocode Nullable(UInt8), -dropoff_boroname Nullable(String), -dropoff_ct2010 Nullable(String), -dropoff_boroct2010 Nullable(String), -dropoff_cdeligibil Nullable(String), -dropoff_ntacode Nullable(String), -dropoff_ntaname Nullable(String), -dropoff_puma Nullable(String) -) ENGINE = Log; -``` - -Es necesario para convertir campos a tipos de datos más correctos y, si es posible, para eliminar NULL. - -``` bash -$ time clickhouse-client --query="INSERT INTO trips FORMAT TabSeparated" < trips.tsv - -real 75m56.214s -``` - -Los datos se leen a una velocidad de 112-140 Mb / segundo. -La carga de datos en una tabla de tipos de registro en una secuencia tardó 76 minutos. -Los datos de esta tabla utilizan 142 GB. - -(Importar datos directamente desde Postgres también es posible usando `COPY ... TO PROGRAM`.) - -Unfortunately, all the fields associated with the weather (precipitation…average_wind_speed) were filled with NULL. Because of this, we will remove them from the final data set. - -Para empezar, crearemos una tabla en un único servidor. Posteriormente haremos la mesa distribuida. - -Crear y rellenar una tabla de resumen: - -``` sql -CREATE TABLE trips_mergetree -ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) -AS SELECT - -trip_id, -CAST(vendor_id AS Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14)) AS vendor_id, -toDate(pickup_datetime) AS pickup_date, -ifNull(pickup_datetime, toDateTime(0)) AS pickup_datetime, -toDate(dropoff_datetime) AS dropoff_date, -ifNull(dropoff_datetime, toDateTime(0)) AS dropoff_datetime, -assumeNotNull(store_and_fwd_flag) IN ('Y', '1', '2') AS store_and_fwd_flag, -assumeNotNull(rate_code_id) AS rate_code_id, -assumeNotNull(pickup_longitude) AS pickup_longitude, -assumeNotNull(pickup_latitude) AS pickup_latitude, -assumeNotNull(dropoff_longitude) AS dropoff_longitude, -assumeNotNull(dropoff_latitude) AS dropoff_latitude, -assumeNotNull(passenger_count) AS passenger_count, -assumeNotNull(trip_distance) AS trip_distance, -assumeNotNull(fare_amount) AS fare_amount, -assumeNotNull(extra) AS extra, -assumeNotNull(mta_tax) AS mta_tax, -assumeNotNull(tip_amount) AS tip_amount, -assumeNotNull(tolls_amount) AS tolls_amount, -assumeNotNull(ehail_fee) AS ehail_fee, -assumeNotNull(improvement_surcharge) AS improvement_surcharge, -assumeNotNull(total_amount) AS total_amount, -CAST((assumeNotNull(payment_type) AS pt) IN ('CSH', 'CASH', 'Cash', 'CAS', 'Cas', '1') ? 'CSH' : (pt IN ('CRD', 'Credit', 'Cre', 'CRE', 'CREDIT', '2') ? 'CRE' : (pt IN ('NOC', 'No Charge', 'No', '3') ? 'NOC' : (pt IN ('DIS', 'Dispute', 'Dis', '4') ? 'DIS' : 'UNK'))) AS Enum8('CSH' = 1, 'CRE' = 2, 'UNK' = 0, 'NOC' = 3, 'DIS' = 4)) AS payment_type_, -assumeNotNull(trip_type) AS trip_type, -ifNull(toFixedString(unhex(pickup), 25), toFixedString('', 25)) AS pickup, -ifNull(toFixedString(unhex(dropoff), 25), toFixedString('', 25)) AS dropoff, -CAST(assumeNotNull(cab_type) AS Enum8('yellow' = 1, 'green' = 2, 'uber' = 3)) AS cab_type, - -assumeNotNull(pickup_nyct2010_gid) AS pickup_nyct2010_gid, -toFloat32(ifNull(pickup_ctlabel, '0')) AS pickup_ctlabel, -assumeNotNull(pickup_borocode) AS pickup_borocode, -CAST(assumeNotNull(pickup_boroname) AS Enum8('Manhattan' = 1, 'Queens' = 4, 'Brooklyn' = 3, '' = 0, 'Bronx' = 2, 'Staten Island' = 5)) AS pickup_boroname, -toFixedString(ifNull(pickup_ct2010, '000000'), 6) AS pickup_ct2010, -toFixedString(ifNull(pickup_boroct2010, '0000000'), 7) AS pickup_boroct2010, -CAST(assumeNotNull(ifNull(pickup_cdeligibil, ' ')) AS Enum8(' ' = 0, 'E' = 1, 'I' = 2)) AS pickup_cdeligibil, -toFixedString(ifNull(pickup_ntacode, '0000'), 4) AS pickup_ntacode, - -CAST(assumeNotNull(pickup_ntaname) AS Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195)) AS pickup_ntaname, - -toUInt16(ifNull(pickup_puma, '0')) AS pickup_puma, - -assumeNotNull(dropoff_nyct2010_gid) AS dropoff_nyct2010_gid, -toFloat32(ifNull(dropoff_ctlabel, '0')) AS dropoff_ctlabel, -assumeNotNull(dropoff_borocode) AS dropoff_borocode, -CAST(assumeNotNull(dropoff_boroname) AS Enum8('Manhattan' = 1, 'Queens' = 4, 'Brooklyn' = 3, '' = 0, 'Bronx' = 2, 'Staten Island' = 5)) AS dropoff_boroname, -toFixedString(ifNull(dropoff_ct2010, '000000'), 6) AS dropoff_ct2010, -toFixedString(ifNull(dropoff_boroct2010, '0000000'), 7) AS dropoff_boroct2010, -CAST(assumeNotNull(ifNull(dropoff_cdeligibil, ' ')) AS Enum8(' ' = 0, 'E' = 1, 'I' = 2)) AS dropoff_cdeligibil, -toFixedString(ifNull(dropoff_ntacode, '0000'), 4) AS dropoff_ntacode, - -CAST(assumeNotNull(dropoff_ntaname) AS Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195)) AS dropoff_ntaname, - -toUInt16(ifNull(dropoff_puma, '0')) AS dropoff_puma - -FROM trips -``` - -Esto toma 3030 segundos a una velocidad de aproximadamente 428,000 filas por segundo. -Para cargarlo más rápido, puede crear la tabla con el `Log` motor en lugar de `MergeTree`. En este caso, la descarga funciona más rápido que 200 segundos. - -La tabla utiliza 126 GB de espacio en disco. - -``` sql -SELECT formatReadableSize(sum(bytes)) FROM system.parts WHERE table = 'trips_mergetree' AND active -``` - -``` text -┌─formatReadableSize(sum(bytes))─┐ -│ 126.18 GiB │ -└────────────────────────────────┘ -``` - -Entre otras cosas, puede ejecutar la consulta OPTIMIZE en MergeTree. Pero no es necesario ya que todo estará bien sin él. - -## Descarga de Prepared Partitions {#download-of-prepared-partitions} - -``` bash -$ curl -O https://datasets.clickhouse.tech/trips_mergetree/partitions/trips_mergetree.tar -$ tar xvf trips_mergetree.tar -C /var/lib/clickhouse # path to ClickHouse data directory -$ # check permissions of unpacked data, fix if required -$ sudo service clickhouse-server restart -$ clickhouse-client --query "select count(*) from datasets.trips_mergetree" -``` - -!!! info "INFO" - Si va a ejecutar las consultas que se describen a continuación, debe usar el nombre completo de la tabla, `datasets.trips_mergetree`. - -## Resultados en un solo servidor {#results-on-single-server} - -Q1: - -``` sql -SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type -``` - -0.490 segundos. - -Q2: - -``` sql -SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count -``` - -1.224 segundos. - -Q3: - -``` sql -SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year -``` - -2.104 segundos. - -Q4: - -``` sql -SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) -FROM trips_mergetree -GROUP BY passenger_count, year, distance -ORDER BY year, count(*) DESC -``` - -3.593 segundos. - -Se utilizó el siguiente servidor: - -Dos CPU Intel (R) Xeon (R) E5-2650 v2 @ 2.60GHz, 16 núcleos físicos en total, 128 GiB RAM, 8x6 TB HD en hardware RAID-5 - -El tiempo de ejecución es el mejor de tres carreras. Pero a partir de la segunda ejecución, las consultas leen datos de la memoria caché del sistema de archivos. No se produce más almacenamiento en caché: los datos se leen y procesan en cada ejecución. - -Creación de una tabla en tres servidores: - -En cada servidor: - -``` sql -CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) -``` - -En el servidor de origen: - -``` sql -CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) -``` - -La siguiente consulta redistribuye los datos: - -``` sql -INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree -``` - -Esto tarda 2454 segundos. - -En tres servidores: - -Q1: 0.212 segundos. -Q2: 0.438 segundos. -Q3: 0.733 segundos. -Q4: 1.241 segundos. - -No hay sorpresas aquí, ya que las consultas se escalan linealmente. - -También tenemos los resultados de un clúster de 140 servidores: - -Q1: 0,028 seg. -Q2: 0,043 seg. -Q3: 0,051 seg. -Q4: 0,072 seg. - -En este caso, el tiempo de procesamiento de la consulta está determinado sobre todo por la latencia de la red. -Ejecutamos consultas utilizando un cliente ubicado en un centro de datos de Yandex en Finlandia en un clúster en Rusia, que agregó aproximadamente 20 ms de latencia. - -## Resumen {#summary} - -| servidor | Q1 | Q2 | Q3 | Q4 | -|----------|-------|-------|-------|-------| -| 1 | 0.490 | 1.224 | 2.104 | 3.593 | -| 3 | 0.212 | 0.438 | 0.733 | 1.241 | -| 140 | 0.028 | 0.043 | 0.051 | 0.072 | - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/nyc_taxi/) diff --git a/docs/es/getting-started/example-datasets/ontime.md b/docs/es/getting-started/example-datasets/ontime.md deleted file mode 100644 index f89d74048bd..00000000000 --- a/docs/es/getting-started/example-datasets/ontime.md +++ /dev/null @@ -1,412 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 15 -toc_title: A tiempo ---- - -# A tiempo {#ontime} - -Este conjunto de datos se puede obtener de dos maneras: - -- importación de datos sin procesar -- descarga de particiones preparadas - -## Importar desde datos sin procesar {#import-from-raw-data} - -Descarga de datos: - -``` bash -for s in `seq 1987 2018` -do -for m in `seq 1 12` -do -wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip -done -done -``` - -(a partir de https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh ) - -Creación de una tabla: - -``` sql -CREATE TABLE `ontime` ( - `Year` UInt16, - `Quarter` UInt8, - `Month` UInt8, - `DayofMonth` UInt8, - `DayOfWeek` UInt8, - `FlightDate` Date, - `UniqueCarrier` FixedString(7), - `AirlineID` Int32, - `Carrier` FixedString(2), - `TailNum` String, - `FlightNum` String, - `OriginAirportID` Int32, - `OriginAirportSeqID` Int32, - `OriginCityMarketID` Int32, - `Origin` FixedString(5), - `OriginCityName` String, - `OriginState` FixedString(2), - `OriginStateFips` String, - `OriginStateName` String, - `OriginWac` Int32, - `DestAirportID` Int32, - `DestAirportSeqID` Int32, - `DestCityMarketID` Int32, - `Dest` FixedString(5), - `DestCityName` String, - `DestState` FixedString(2), - `DestStateFips` String, - `DestStateName` String, - `DestWac` Int32, - `CRSDepTime` Int32, - `DepTime` Int32, - `DepDelay` Int32, - `DepDelayMinutes` Int32, - `DepDel15` Int32, - `DepartureDelayGroups` String, - `DepTimeBlk` String, - `TaxiOut` Int32, - `WheelsOff` Int32, - `WheelsOn` Int32, - `TaxiIn` Int32, - `CRSArrTime` Int32, - `ArrTime` Int32, - `ArrDelay` Int32, - `ArrDelayMinutes` Int32, - `ArrDel15` Int32, - `ArrivalDelayGroups` Int32, - `ArrTimeBlk` String, - `Cancelled` UInt8, - `CancellationCode` FixedString(1), - `Diverted` UInt8, - `CRSElapsedTime` Int32, - `ActualElapsedTime` Int32, - `AirTime` Int32, - `Flights` Int32, - `Distance` Int32, - `DistanceGroup` UInt8, - `CarrierDelay` Int32, - `WeatherDelay` Int32, - `NASDelay` Int32, - `SecurityDelay` Int32, - `LateAircraftDelay` Int32, - `FirstDepTime` String, - `TotalAddGTime` String, - `LongestAddGTime` String, - `DivAirportLandings` String, - `DivReachedDest` String, - `DivActualElapsedTime` String, - `DivArrDelay` String, - `DivDistance` String, - `Div1Airport` String, - `Div1AirportID` Int32, - `Div1AirportSeqID` Int32, - `Div1WheelsOn` String, - `Div1TotalGTime` String, - `Div1LongestGTime` String, - `Div1WheelsOff` String, - `Div1TailNum` String, - `Div2Airport` String, - `Div2AirportID` Int32, - `Div2AirportSeqID` Int32, - `Div2WheelsOn` String, - `Div2TotalGTime` String, - `Div2LongestGTime` String, - `Div2WheelsOff` String, - `Div2TailNum` String, - `Div3Airport` String, - `Div3AirportID` Int32, - `Div3AirportSeqID` Int32, - `Div3WheelsOn` String, - `Div3TotalGTime` String, - `Div3LongestGTime` String, - `Div3WheelsOff` String, - `Div3TailNum` String, - `Div4Airport` String, - `Div4AirportID` Int32, - `Div4AirportSeqID` Int32, - `Div4WheelsOn` String, - `Div4TotalGTime` String, - `Div4LongestGTime` String, - `Div4WheelsOff` String, - `Div4TailNum` String, - `Div5Airport` String, - `Div5AirportID` Int32, - `Div5AirportSeqID` Int32, - `Div5WheelsOn` String, - `Div5TotalGTime` String, - `Div5LongestGTime` String, - `Div5WheelsOff` String, - `Div5TailNum` String -) ENGINE = MergeTree -PARTITION BY Year -ORDER BY (Carrier, FlightDate) -SETTINGS index_granularity = 8192; -``` - -Carga de datos: - -``` bash -$ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done -``` - -## Descarga de Prepared Partitions {#download-of-prepared-partitions} - -``` bash -$ curl -O https://datasets.clickhouse.tech/ontime/partitions/ontime.tar -$ tar xvf ontime.tar -C /var/lib/clickhouse # path to ClickHouse data directory -$ # check permissions of unpacked data, fix if required -$ sudo service clickhouse-server restart -$ clickhouse-client --query "select count(*) from datasets.ontime" -``` - -!!! info "INFO" - Si va a ejecutar las consultas que se describen a continuación, debe usar el nombre completo de la tabla, `datasets.ontime`. - -## Consulta {#queries} - -Q0. - -``` sql -SELECT avg(c1) -FROM -( - SELECT Year, Month, count(*) AS c1 - FROM ontime - GROUP BY Year, Month -); -``` - -Q1. El número de vuelos por día desde el año 2000 hasta 2008 - -``` sql -SELECT DayOfWeek, count(*) AS c -FROM ontime -WHERE Year>=2000 AND Year<=2008 -GROUP BY DayOfWeek -ORDER BY c DESC; -``` - -Preguntas frecuentes El número de vuelos retrasados por más de 10 minutos, agrupados por el día de la semana, para 2000-2008 - -``` sql -SELECT DayOfWeek, count(*) AS c -FROM ontime -WHERE DepDelay>10 AND Year>=2000 AND Year<=2008 -GROUP BY DayOfWeek -ORDER BY c DESC; -``` - -Q3. El número de retrasos por parte del aeropuerto para 2000-2008 - -``` sql -SELECT Origin, count(*) AS c -FROM ontime -WHERE DepDelay>10 AND Year>=2000 AND Year<=2008 -GROUP BY Origin -ORDER BY c DESC -LIMIT 10; -``` - -Preguntas más frecuentes Número de retrasos por transportista para 2007 - -``` sql -SELECT Carrier, count(*) -FROM ontime -WHERE DepDelay>10 AND Year=2007 -GROUP BY Carrier -ORDER BY count(*) DESC; -``` - -Q5. El porcentaje de retrasos por transportista para 2007 - -``` sql -SELECT Carrier, c, c2, c*100/c2 as c3 -FROM -( - SELECT - Carrier, - count(*) AS c - FROM ontime - WHERE DepDelay>10 - AND Year=2007 - GROUP BY Carrier -) -JOIN -( - SELECT - Carrier, - count(*) AS c2 - FROM ontime - WHERE Year=2007 - GROUP BY Carrier -) USING Carrier -ORDER BY c3 DESC; -``` - -Mejor versión de la misma consulta: - -``` sql -SELECT Carrier, avg(DepDelay>10)*100 AS c3 -FROM ontime -WHERE Year=2007 -GROUP BY Carrier -ORDER BY c3 DESC -``` - -¿Por qué? La solicitud anterior de una gama más amplia de años, 2000-2008 - -``` sql -SELECT Carrier, c, c2, c*100/c2 as c3 -FROM -( - SELECT - Carrier, - count(*) AS c - FROM ontime - WHERE DepDelay>10 - AND Year>=2000 AND Year<=2008 - GROUP BY Carrier -) -JOIN -( - SELECT - Carrier, - count(*) AS c2 - FROM ontime - WHERE Year>=2000 AND Year<=2008 - GROUP BY Carrier -) USING Carrier -ORDER BY c3 DESC; -``` - -Mejor versión de la misma consulta: - -``` sql -SELECT Carrier, avg(DepDelay>10)*100 AS c3 -FROM ontime -WHERE Year>=2000 AND Year<=2008 -GROUP BY Carrier -ORDER BY c3 DESC; -``` - -Preguntas frecuentes Porcentaje de vuelos retrasados por más de 10 minutos, por año - -``` sql -SELECT Year, c1/c2 -FROM -( - select - Year, - count(*)*100 as c1 - from ontime - WHERE DepDelay>10 - GROUP BY Year -) -JOIN -( - select - Year, - count(*) as c2 - from ontime - GROUP BY Year -) USING (Year) -ORDER BY Year; -``` - -Mejor versión de la misma consulta: - -``` sql -SELECT Year, avg(DepDelay>10)*100 -FROM ontime -GROUP BY Year -ORDER BY Year; -``` - -¿Por qué? Los destinos más populares por el número de ciudades conectadas directamente para varios rangos de año - -``` sql -SELECT DestCityName, uniqExact(OriginCityName) AS u -FROM ontime -WHERE Year >= 2000 and Year <= 2010 -GROUP BY DestCityName -ORDER BY u DESC LIMIT 10; -``` - -Q9. - -``` sql -SELECT Year, count(*) AS c1 -FROM ontime -GROUP BY Year; -``` - -Q10. - -``` sql -SELECT - min(Year), max(Year), Carrier, count(*) AS cnt, - sum(ArrDelayMinutes>30) AS flights_delayed, - round(sum(ArrDelayMinutes>30)/count(*),2) AS rate -FROM ontime -WHERE - DayOfWeek NOT IN (6,7) AND OriginState NOT IN ('AK', 'HI', 'PR', 'VI') - AND DestState NOT IN ('AK', 'HI', 'PR', 'VI') - AND FlightDate < '2010-01-01' -GROUP by Carrier -HAVING cnt>100000 and max(Year)>1990 -ORDER by rate DESC -LIMIT 1000; -``` - -Bono: - -``` sql -SELECT avg(cnt) -FROM -( - SELECT Year,Month,count(*) AS cnt - FROM ontime - WHERE DepDel15=1 - GROUP BY Year,Month -); - -SELECT avg(c1) FROM -( - SELECT Year,Month,count(*) AS c1 - FROM ontime - GROUP BY Year,Month -); - -SELECT DestCityName, uniqExact(OriginCityName) AS u -FROM ontime -GROUP BY DestCityName -ORDER BY u DESC -LIMIT 10; - -SELECT OriginCityName, DestCityName, count() AS c -FROM ontime -GROUP BY OriginCityName, DestCityName -ORDER BY c DESC -LIMIT 10; - -SELECT OriginCityName, count() AS c -FROM ontime -GROUP BY OriginCityName -ORDER BY c DESC -LIMIT 10; -``` - -Esta prueba de rendimiento fue creada por Vadim Tkachenko. Ver: - -- https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/ -- https://www.percona.com/blog/2009/10/26/air-traffic-queries-in-luciddb/ -- https://www.percona.com/blog/2009/11/02/air-traffic-queries-in-infinidb-early-alpha/ -- https://www.percona.com/blog/2014/04/21/using-apache-hadoop-and-impala-together-with-mysql-for-data-analysis/ -- https://www.percona.com/blog/2016/01/07/apache-spark-with-air-ontime-performance-data/ -- http://nickmakos.blogspot.ru/2012/08/analyzing-air-traffic-performance-with.html - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/ontime/) diff --git a/docs/es/getting-started/example-datasets/star-schema.md b/docs/es/getting-started/example-datasets/star-schema.md deleted file mode 100644 index 43f878eb205..00000000000 --- a/docs/es/getting-started/example-datasets/star-schema.md +++ /dev/null @@ -1,370 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 20 -toc_title: Estrella Schema Benchmark ---- - -# Estrella Schema Benchmark {#star-schema-benchmark} - -Compilación de dbgen: - -``` bash -$ git clone git@github.com:vadimtk/ssb-dbgen.git -$ cd ssb-dbgen -$ make -``` - -Generación de datos: - -!!! warning "Atención" - Con `-s 100` dbgen genera 600 millones de filas (67 GB), mientras que `-s 1000` genera 6 mil millones de filas (lo que lleva mucho tiempo) - -``` bash -$ ./dbgen -s 1000 -T c -$ ./dbgen -s 1000 -T l -$ ./dbgen -s 1000 -T p -$ ./dbgen -s 1000 -T s -$ ./dbgen -s 1000 -T d -``` - -Creación de tablas en ClickHouse: - -``` sql -CREATE TABLE customer -( - C_CUSTKEY UInt32, - C_NAME String, - C_ADDRESS String, - C_CITY LowCardinality(String), - C_NATION LowCardinality(String), - C_REGION LowCardinality(String), - C_PHONE String, - C_MKTSEGMENT LowCardinality(String) -) -ENGINE = MergeTree ORDER BY (C_CUSTKEY); - -CREATE TABLE lineorder -( - LO_ORDERKEY UInt32, - LO_LINENUMBER UInt8, - LO_CUSTKEY UInt32, - LO_PARTKEY UInt32, - LO_SUPPKEY UInt32, - LO_ORDERDATE Date, - LO_ORDERPRIORITY LowCardinality(String), - LO_SHIPPRIORITY UInt8, - LO_QUANTITY UInt8, - LO_EXTENDEDPRICE UInt32, - LO_ORDTOTALPRICE UInt32, - LO_DISCOUNT UInt8, - LO_REVENUE UInt32, - LO_SUPPLYCOST UInt32, - LO_TAX UInt8, - LO_COMMITDATE Date, - LO_SHIPMODE LowCardinality(String) -) -ENGINE = MergeTree PARTITION BY toYear(LO_ORDERDATE) ORDER BY (LO_ORDERDATE, LO_ORDERKEY); - -CREATE TABLE part -( - P_PARTKEY UInt32, - P_NAME String, - P_MFGR LowCardinality(String), - P_CATEGORY LowCardinality(String), - P_BRAND LowCardinality(String), - P_COLOR LowCardinality(String), - P_TYPE LowCardinality(String), - P_SIZE UInt8, - P_CONTAINER LowCardinality(String) -) -ENGINE = MergeTree ORDER BY P_PARTKEY; - -CREATE TABLE supplier -( - S_SUPPKEY UInt32, - S_NAME String, - S_ADDRESS String, - S_CITY LowCardinality(String), - S_NATION LowCardinality(String), - S_REGION LowCardinality(String), - S_PHONE String -) -ENGINE = MergeTree ORDER BY S_SUPPKEY; -``` - -Insertar datos: - -``` bash -$ clickhouse-client --query "INSERT INTO customer FORMAT CSV" < customer.tbl -$ clickhouse-client --query "INSERT INTO part FORMAT CSV" < part.tbl -$ clickhouse-client --query "INSERT INTO supplier FORMAT CSV" < supplier.tbl -$ clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" < lineorder.tbl -``` - -Conversión “star schema” a desnormalizado “flat schema”: - -``` sql -SET max_memory_usage = 20000000000; - -CREATE TABLE lineorder_flat -ENGINE = MergeTree -PARTITION BY toYear(LO_ORDERDATE) -ORDER BY (LO_ORDERDATE, LO_ORDERKEY) AS -SELECT - l.LO_ORDERKEY AS LO_ORDERKEY, - l.LO_LINENUMBER AS LO_LINENUMBER, - l.LO_CUSTKEY AS LO_CUSTKEY, - l.LO_PARTKEY AS LO_PARTKEY, - l.LO_SUPPKEY AS LO_SUPPKEY, - l.LO_ORDERDATE AS LO_ORDERDATE, - l.LO_ORDERPRIORITY AS LO_ORDERPRIORITY, - l.LO_SHIPPRIORITY AS LO_SHIPPRIORITY, - l.LO_QUANTITY AS LO_QUANTITY, - l.LO_EXTENDEDPRICE AS LO_EXTENDEDPRICE, - l.LO_ORDTOTALPRICE AS LO_ORDTOTALPRICE, - l.LO_DISCOUNT AS LO_DISCOUNT, - l.LO_REVENUE AS LO_REVENUE, - l.LO_SUPPLYCOST AS LO_SUPPLYCOST, - l.LO_TAX AS LO_TAX, - l.LO_COMMITDATE AS LO_COMMITDATE, - l.LO_SHIPMODE AS LO_SHIPMODE, - c.C_NAME AS C_NAME, - c.C_ADDRESS AS C_ADDRESS, - c.C_CITY AS C_CITY, - c.C_NATION AS C_NATION, - c.C_REGION AS C_REGION, - c.C_PHONE AS C_PHONE, - c.C_MKTSEGMENT AS C_MKTSEGMENT, - s.S_NAME AS S_NAME, - s.S_ADDRESS AS S_ADDRESS, - s.S_CITY AS S_CITY, - s.S_NATION AS S_NATION, - s.S_REGION AS S_REGION, - s.S_PHONE AS S_PHONE, - p.P_NAME AS P_NAME, - p.P_MFGR AS P_MFGR, - p.P_CATEGORY AS P_CATEGORY, - p.P_BRAND AS P_BRAND, - p.P_COLOR AS P_COLOR, - p.P_TYPE AS P_TYPE, - p.P_SIZE AS P_SIZE, - p.P_CONTAINER AS P_CONTAINER -FROM lineorder AS l -INNER JOIN customer AS c ON c.C_CUSTKEY = l.LO_CUSTKEY -INNER JOIN supplier AS s ON s.S_SUPPKEY = l.LO_SUPPKEY -INNER JOIN part AS p ON p.P_PARTKEY = l.LO_PARTKEY; -``` - -Las consultas: - -Q1.1 - -``` sql -SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue -FROM lineorder_flat -WHERE toYear(LO_ORDERDATE) = 1993 AND LO_DISCOUNT BETWEEN 1 AND 3 AND LO_QUANTITY < 25; -``` - -Q1.2 - -``` sql -SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue -FROM lineorder_flat -WHERE toYYYYMM(LO_ORDERDATE) = 199401 AND LO_DISCOUNT BETWEEN 4 AND 6 AND LO_QUANTITY BETWEEN 26 AND 35; -``` - -Q1.3 - -``` sql -SELECT sum(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue -FROM lineorder_flat -WHERE toISOWeek(LO_ORDERDATE) = 6 AND toYear(LO_ORDERDATE) = 1994 - AND LO_DISCOUNT BETWEEN 5 AND 7 AND LO_QUANTITY BETWEEN 26 AND 35; -``` - -Q2.1 - -``` sql -SELECT - sum(LO_REVENUE), - toYear(LO_ORDERDATE) AS year, - P_BRAND -FROM lineorder_flat -WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' -GROUP BY - year, - P_BRAND -ORDER BY - year, - P_BRAND; -``` - -Q2.2 - -``` sql -SELECT - sum(LO_REVENUE), - toYear(LO_ORDERDATE) AS year, - P_BRAND -FROM lineorder_flat -WHERE P_BRAND >= 'MFGR#2221' AND P_BRAND <= 'MFGR#2228' AND S_REGION = 'ASIA' -GROUP BY - year, - P_BRAND -ORDER BY - year, - P_BRAND; -``` - -Q2.3 - -``` sql -SELECT - sum(LO_REVENUE), - toYear(LO_ORDERDATE) AS year, - P_BRAND -FROM lineorder_flat -WHERE P_BRAND = 'MFGR#2239' AND S_REGION = 'EUROPE' -GROUP BY - year, - P_BRAND -ORDER BY - year, - P_BRAND; -``` - -Q3.1 - -``` sql -SELECT - C_NATION, - S_NATION, - toYear(LO_ORDERDATE) AS year, - sum(LO_REVENUE) AS revenue -FROM lineorder_flat -WHERE C_REGION = 'ASIA' AND S_REGION = 'ASIA' AND year >= 1992 AND year <= 1997 -GROUP BY - C_NATION, - S_NATION, - year -ORDER BY - year ASC, - revenue DESC; -``` - -Q3.2 - -``` sql -SELECT - C_CITY, - S_CITY, - toYear(LO_ORDERDATE) AS year, - sum(LO_REVENUE) AS revenue -FROM lineorder_flat -WHERE C_NATION = 'UNITED STATES' AND S_NATION = 'UNITED STATES' AND year >= 1992 AND year <= 1997 -GROUP BY - C_CITY, - S_CITY, - year -ORDER BY - year ASC, - revenue DESC; -``` - -Q3.3 - -``` sql -SELECT - C_CITY, - S_CITY, - toYear(LO_ORDERDATE) AS year, - sum(LO_REVENUE) AS revenue -FROM lineorder_flat -WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND year >= 1992 AND year <= 1997 -GROUP BY - C_CITY, - S_CITY, - year -ORDER BY - year ASC, - revenue DESC; -``` - -Q3.4 - -``` sql -SELECT - C_CITY, - S_CITY, - toYear(LO_ORDERDATE) AS year, - sum(LO_REVENUE) AS revenue -FROM lineorder_flat -WHERE (C_CITY = 'UNITED KI1' OR C_CITY = 'UNITED KI5') AND (S_CITY = 'UNITED KI1' OR S_CITY = 'UNITED KI5') AND toYYYYMM(LO_ORDERDATE) = 199712 -GROUP BY - C_CITY, - S_CITY, - year -ORDER BY - year ASC, - revenue DESC; -``` - -Q4.1 - -``` sql -SELECT - toYear(LO_ORDERDATE) AS year, - C_NATION, - sum(LO_REVENUE - LO_SUPPLYCOST) AS profit -FROM lineorder_flat -WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') -GROUP BY - year, - C_NATION -ORDER BY - year ASC, - C_NATION ASC; -``` - -Q4.2 - -``` sql -SELECT - toYear(LO_ORDERDATE) AS year, - S_NATION, - P_CATEGORY, - sum(LO_REVENUE - LO_SUPPLYCOST) AS profit -FROM lineorder_flat -WHERE C_REGION = 'AMERICA' AND S_REGION = 'AMERICA' AND (year = 1997 OR year = 1998) AND (P_MFGR = 'MFGR#1' OR P_MFGR = 'MFGR#2') -GROUP BY - year, - S_NATION, - P_CATEGORY -ORDER BY - year ASC, - S_NATION ASC, - P_CATEGORY ASC; -``` - -Q4.3 - -``` sql -SELECT - toYear(LO_ORDERDATE) AS year, - S_CITY, - P_BRAND, - sum(LO_REVENUE - LO_SUPPLYCOST) AS profit -FROM lineorder_flat -WHERE S_NATION = 'UNITED STATES' AND (year = 1997 OR year = 1998) AND P_CATEGORY = 'MFGR#14' -GROUP BY - year, - S_CITY, - P_BRAND -ORDER BY - year ASC, - S_CITY ASC, - P_BRAND ASC; -``` - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/star_schema/) diff --git a/docs/es/getting-started/example-datasets/wikistat.md b/docs/es/getting-started/example-datasets/wikistat.md deleted file mode 100644 index 49d7263cdec..00000000000 --- a/docs/es/getting-started/example-datasets/wikistat.md +++ /dev/null @@ -1,35 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 18 -toc_title: "Nombre de la red inal\xE1mbrica (SSID):" ---- - -# Nombre de la red inalámbrica (SSID): {#wikistat} - -Ver: http://dumps.wikimedia.org/other/pagecounts-raw/ - -Creación de una tabla: - -``` sql -CREATE TABLE wikistat -( - date Date, - time DateTime, - project String, - subproject String, - path String, - hits UInt64, - size UInt64 -) ENGINE = MergeTree(date, (path, time), 8192); -``` - -Carga de datos: - -``` bash -$ for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt -$ cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done -$ ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done -``` - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/example_datasets/wikistat/) diff --git a/docs/es/getting-started/index.md b/docs/es/getting-started/index.md deleted file mode 100644 index 681c2017ac1..00000000000 --- a/docs/es/getting-started/index.md +++ /dev/null @@ -1,17 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Primeros pasos -toc_hidden: true -toc_priority: 8 -toc_title: oculto ---- - -# Primeros pasos {#getting-started} - -Si eres nuevo en ClickHouse y quieres tener una sensación práctica de su rendimiento, antes que nada, debes pasar por el [proceso de instalación](install.md). Después de eso puedes: - -- [Ir a través de tutorial detallado](tutorial.md) -- [Experimente con conjuntos de datos de ejemplo](example-datasets/ontime.md) - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/) diff --git a/docs/es/getting-started/install.md b/docs/es/getting-started/install.md deleted file mode 100644 index 092ef47b2f7..00000000000 --- a/docs/es/getting-started/install.md +++ /dev/null @@ -1,182 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 11 -toc_title: "Instalaci\xF3n" ---- - -# Instalación {#installation} - -## Requisitos del sistema {#system-requirements} - -ClickHouse puede ejecutarse en cualquier Linux, FreeBSD o Mac OS X con arquitectura de CPU x86_64, AArch64 o PowerPC64LE. - -Los binarios oficiales preconstruidos generalmente se compilan para x86_64 y aprovechan el conjunto de instrucciones SSE 4.2, por lo que, a menos que se indique lo contrario, el uso de la CPU que lo admite se convierte en un requisito adicional del sistema. Aquí está el comando para verificar si la CPU actual tiene soporte para SSE 4.2: - -``` bash -$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" -``` - -Para ejecutar ClickHouse en procesadores que no admiten SSE 4.2 o tienen arquitectura AArch64 o PowerPC64LE, debe [construir ClickHouse a partir de fuentes](#from-sources) con los ajustes de configuración adecuados. - -## Opciones de instalación disponibles {#available-installation-options} - -### De paquetes DEB {#install-from-deb-packages} - -Se recomienda utilizar pre-compilado oficial `deb` Paquetes para Debian o Ubuntu. Ejecute estos comandos para instalar paquetes: - -``` bash -{% include 'install/deb.sh' %} -``` - -Si desea utilizar la versión más reciente, reemplace `stable` con `testing` (esto se recomienda para sus entornos de prueba). - -También puede descargar e instalar paquetes manualmente desde [aqui](https://repo.clickhouse.tech/deb/stable/main/). - -#### Paquete {#packages} - -- `clickhouse-common-static` — Installs ClickHouse compiled binary files. -- `clickhouse-server` — Creates a symbolic link for `clickhouse-server` e instala la configuración predeterminada del servidor. -- `clickhouse-client` — Creates a symbolic link for `clickhouse-client` y otras herramientas relacionadas con el cliente. e instala los archivos de configuración del cliente. -- `clickhouse-common-static-dbg` — Installs ClickHouse compiled binary files with debug info. - -### De paquetes RPM {#from-rpm-packages} - -Se recomienda utilizar pre-compilado oficial `rpm` También puede utilizar los paquetes para CentOS, RedHat y todas las demás distribuciones de Linux basadas en rpm. - -Primero, necesitas agregar el repositorio oficial: - -``` bash -sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 -``` - -Si desea utilizar la versión más reciente, reemplace `stable` con `testing` (esto se recomienda para sus entornos de prueba). El `prestable` etiqueta a veces está disponible también. - -A continuación, ejecute estos comandos para instalar paquetes: - -``` bash -sudo yum install clickhouse-server clickhouse-client -``` - -También puede descargar e instalar paquetes manualmente desde [aqui](https://repo.clickhouse.tech/rpm/stable/x86_64). - -### De archivos Tgz {#from-tgz-archives} - -Se recomienda utilizar pre-compilado oficial `tgz` para todas las distribuciones de Linux, donde la instalación de `deb` o `rpm` paquetes no es posible. - -La versión requerida se puede descargar con `curl` o `wget` desde el repositorio https://repo.clickhouse.tech/tgz/. -Después de eso, los archivos descargados deben desempaquetarse e instalarse con scripts de instalación. Ejemplo para la última versión: - -``` bash -export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-client-$LATEST_VERSION.tgz - -tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz -sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-common-static-dbg-$LATEST_VERSION.tgz -sudo clickhouse-common-static-dbg-$LATEST_VERSION/install/doinst.sh - -tar -xzvf clickhouse-server-$LATEST_VERSION.tgz -sudo clickhouse-server-$LATEST_VERSION/install/doinst.sh -sudo /etc/init.d/clickhouse-server start - -tar -xzvf clickhouse-client-$LATEST_VERSION.tgz -sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh -``` - -Para los entornos de producción, se recomienda utilizar las últimas `stable`-versión. Puede encontrar su número en la página de GitHub https://github.com/ClickHouse/ClickHouse/tags con postfix `-stable`. - -### Desde Docker Image {#from-docker-image} - -Para ejecutar ClickHouse dentro de Docker, siga la guía en [Eje de acoplador](https://hub.docker.com/r/yandex/clickhouse-server/). Esas imágenes usan oficial `deb` paquetes dentro. - -### De fuentes {#from-sources} - -Para compilar manualmente ClickHouse, siga las instrucciones para [Linux](../development/build.md) o [Mac OS X](../development/build-osx.md). - -Puede compilar paquetes e instalarlos o usar programas sin instalar paquetes. Además, al construir manualmente, puede deshabilitar el requisito de SSE 4.2 o compilar para CPU AArch64. - - Client: programs/clickhouse-client - Server: programs/clickhouse-server - -Tendrá que crear carpetas de datos y metadatos y `chown` para el usuario deseado. Sus rutas se pueden cambiar en la configuración del servidor (src/programs/server/config.xml), por defecto son: - - /opt/clickhouse/data/default/ - /opt/clickhouse/metadata/default/ - -En Gentoo, puedes usar `emerge clickhouse` para instalar ClickHouse desde fuentes. - -## Lanzar {#launch} - -Para iniciar el servidor como demonio, ejecute: - -``` bash -$ sudo service clickhouse-server start -``` - -Si no tienes `service` comando ejecutar como - -``` bash -$ sudo /etc/init.d/clickhouse-server start -``` - -Vea los registros en el `/var/log/clickhouse-server/` directorio. - -Si el servidor no se inicia, compruebe las configuraciones en el archivo `/etc/clickhouse-server/config.xml`. - -También puede iniciar manualmente el servidor desde la consola: - -``` bash -$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml -``` - -En este caso, el registro se imprimirá en la consola, lo cual es conveniente durante el desarrollo. -Si el archivo de configuración está en el directorio actual, no es necesario `--config-file` parámetro. De forma predeterminada, utiliza `./config.xml`. - -ClickHouse admite la configuración de restricción de acceso. Están ubicados en el `users.xml` archivo (junto a `config.xml`). -De forma predeterminada, se permite el acceso desde cualquier lugar `default` usuario, sin una contraseña. Ver `user/default/networks`. -Para obtener más información, consulte la sección [“Configuration Files”](../operations/configuration-files.md). - -Después de iniciar el servidor, puede usar el cliente de línea de comandos para conectarse a él: - -``` bash -$ clickhouse-client -``` - -Por defecto, se conecta a `localhost:9000` en nombre del usuario `default` sin una contraseña. También se puede usar para conectarse a un servidor remoto usando `--host` argumento. - -El terminal debe usar codificación UTF-8. -Para obtener más información, consulte la sección [“Command-line client”](../interfaces/cli.md). - -Ejemplo: - -``` bash -$ ./clickhouse-client -ClickHouse client version 0.0.18749. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.18749. - -:) SELECT 1 - -SELECT 1 - -┌─1─┐ -│ 1 │ -└───┘ - -1 rows in set. Elapsed: 0.003 sec. - -:) -``` - -**Felicidades, el sistema funciona!** - -Para continuar experimentando, puede descargar uno de los conjuntos de datos de prueba o pasar por [tutorial](https://clickhouse.tech/tutorial.html). - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/install/) diff --git a/docs/es/getting-started/playground.md b/docs/es/getting-started/playground.md deleted file mode 100644 index 1ab7246e2d4..00000000000 --- a/docs/es/getting-started/playground.md +++ /dev/null @@ -1,48 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 14 -toc_title: Infantil ---- - -# Zona de juegos ClickHouse {#clickhouse-playground} - -[Zona de juegos ClickHouse](https://play.clickhouse.tech?file=welcome) permite a las personas experimentar con ClickHouse ejecutando consultas al instante, sin configurar su servidor o clúster. -Varios conjuntos de datos de ejemplo están disponibles en Playground, así como consultas de ejemplo que muestran las características de ClickHouse. - -Las consultas se ejecutan como un usuario de sólo lectura. Implica algunas limitaciones: - -- No se permiten consultas DDL -- Las consultas INSERT no están permitidas - -También se aplican los siguientes valores: -- [`max_result_bytes=10485760`](../operations/settings/query_complexity/#max-result-bytes) -- [`max_result_rows=2000`](../operations/settings/query_complexity/#setting-max_result_rows) -- [`result_overflow_mode=break`](../operations/settings/query_complexity/#result-overflow-mode) -- [`max_execution_time=60000`](../operations/settings/query_complexity/#max-execution-time) - -ClickHouse Playground da la experiencia de m2.pequeño -[Servicio administrado para ClickHouse](https://cloud.yandex.com/services/managed-clickhouse) -instancia alojada en [El Yandex.Nube](https://cloud.yandex.com/). -Más información sobre [proveedores de la nube](../commercial/cloud.md). - -La interfaz web de ClickHouse Playground realiza solicitudes a través de ClickHouse [HTTP API](../interfaces/http.md). -El backend Playground es solo un clúster ClickHouse sin ninguna aplicación adicional del lado del servidor. -El punto final HTTPS de ClickHouse también está disponible como parte de Playground. - -Puede realizar consultas al patio de recreo utilizando cualquier cliente HTTP, por ejemplo [rizo](https://curl.haxx.se) o [wget](https://www.gnu.org/software/wget/), o configurar una conexión usando [JDBC](../interfaces/jdbc.md) o [ODBC](../interfaces/odbc.md) controlador. -Más información sobre los productos de software compatibles con ClickHouse está disponible [aqui](../interfaces/index.md). - -| Parámetro | Valor | -|:------------|:----------------------------------------------| -| Punto final | https://play-api.casa de clic.tecnología:8443 | -| Usuario | `playground` | -| Contraseña | `clickhouse` | - -Tenga en cuenta que este extremo requiere una conexión segura. - -Ejemplo: - -``` bash -curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse!';&user=playground&password=clickhouse&database=datasets" -``` diff --git a/docs/es/getting-started/tutorial.md b/docs/es/getting-started/tutorial.md deleted file mode 100644 index 2cc9339f954..00000000000 --- a/docs/es/getting-started/tutorial.md +++ /dev/null @@ -1,664 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 12 -toc_title: Tutorial ---- - -# Tutorial de ClickHouse {#clickhouse-tutorial} - -## Qué Esperar de Este Tutorial? {#what-to-expect-from-this-tutorial} - -Al pasar por este tutorial, aprenderá cómo configurar un clúster de ClickHouse simple. Será pequeño, pero tolerante a fallos y escalable. Luego usaremos uno de los conjuntos de datos de ejemplo para llenarlo con datos y ejecutar algunas consultas de demostración. - -## Configuración de nodo único {#single-node-setup} - -Para posponer las complejidades de un entorno distribuido, comenzaremos con la implementación de ClickHouse en un único servidor o máquina virtual. ClickHouse generalmente se instala desde [deb](install.md#install-from-deb-packages) o [RPM](install.md#from-rpm-packages) paquetes, pero hay [alternativa](install.md#from-docker-image) para los sistemas operativos que no los admiten. - -Por ejemplo, ha elegido `deb` paquetes y ejecutado: - -``` bash -{% include 'install/deb.sh' %} -``` - -¿Qué tenemos en los paquetes que tengo instalados: - -- `clickhouse-client` el paquete contiene [Casa de clics-cliente](../interfaces/cli.md) aplicación, cliente interactivo de la consola ClickHouse. -- `clickhouse-common` El paquete contiene un archivo ejecutable ClickHouse. -- `clickhouse-server` El paquete contiene archivos de configuración para ejecutar ClickHouse como servidor. - -Los archivos de configuración del servidor se encuentran en `/etc/clickhouse-server/`. Antes de ir más lejos, tenga en cuenta el `` elemento en `config.xml`. La ruta determina la ubicación para el almacenamiento de datos, por lo que debe ubicarse en un volumen con gran capacidad de disco; el valor predeterminado es `/var/lib/clickhouse/`. Si desea ajustar la configuración, no es útil editar directamente `config.xml` archivo, teniendo en cuenta que podría ser reescrito en futuras actualizaciones de paquetes. La forma recomendada de anular los elementos de configuración es crear [archivos en config.directorio d](../operations/configuration-files.md) que sirven como “patches” de configuración.XML. - -Como habrás notado, `clickhouse-server` no se inicia automáticamente después de la instalación del paquete. Tampoco se reiniciará automáticamente después de las actualizaciones. La forma en que inicia el servidor depende de su sistema de inicio, por lo general, es: - -``` bash -sudo service clickhouse-server start -``` - -o - -``` bash -sudo /etc/init.d/clickhouse-server start -``` - -La ubicación predeterminada para los registros del servidor es `/var/log/clickhouse-server/`. El servidor está listo para manejar las conexiones de cliente una vez que registra el `Ready for connections` mensaje. - -Una vez que el `clickhouse-server` está en funcionamiento, podemos usar `clickhouse-client` para conectarse al servidor y ejecutar algunas consultas de prueba como `SELECT "Hello, world!";`. - -
- -Consejos rápidos para clickhouse-cliente - -Modo interactivo: - -``` bash -clickhouse-client -clickhouse-client --host=... --port=... --user=... --password=... -``` - -Habilitar consultas multilínea: - -``` bash -clickhouse-client -m -clickhouse-client --multiline -``` - -Ejecutar consultas en modo por lotes: - -``` bash -clickhouse-client --query='SELECT 1' -echo 'SELECT 1' | clickhouse-client -clickhouse-client <<< 'SELECT 1' -``` - -Insertar datos de un archivo en el formato especificado: - -``` bash -clickhouse-client --query='INSERT INTO table VALUES' < data.txt -clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv -``` - -
- -## Importar conjunto de datos de muestra {#import-sample-dataset} - -Ahora es el momento de llenar nuestro servidor ClickHouse con algunos datos de muestra. En este tutorial, usaremos los datos anónimos de Yandex.Metrica, el primer servicio que ejecuta ClickHouse en forma de producción antes de que se convirtiera en código abierto (más sobre eso en [sección de historia](../introduction/history.md)). Hay [múltiples formas de importar Yandex.Conjunto de datos de Metrica](example-datasets/metrica.md), y por el bien del tutorial, iremos con el más realista. - -### Descargar y extraer datos de tabla {#download-and-extract-table-data} - -``` bash -curl https://datasets.clickhouse.tech/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv -curl https://datasets.clickhouse.tech/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv -``` - -Los archivos extraídos tienen un tamaño de aproximadamente 10 GB. - -### Crear tablas {#create-tables} - -Como en la mayoría de los sistemas de gestión de bases de datos, ClickHouse agrupa lógicamente las tablas en “databases”. Hay un `default` base de datos, pero crearemos una nueva llamada `tutorial`: - -``` bash -clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial" -``` - -La sintaxis para crear tablas es mucho más complicada en comparación con las bases de datos (ver [referencia](../sql-reference/statements/create.md). En general `CREATE TABLE` declaración tiene que especificar tres cosas clave: - -1. Nombre de la tabla que se va a crear. -2. Table schema, i.e. list of columns and their [tipos de datos](../sql-reference/data-types/index.md). -3. [Motor de tabla](../engines/table-engines/index.md) y su configuración, que determina todos los detalles sobre cómo se ejecutarán físicamente las consultas a esta tabla. - -El Yandex.Metrica es un servicio de análisis web, y el conjunto de datos de muestra no cubre toda su funcionalidad, por lo que solo hay dos tablas para crear: - -- `hits` es una tabla con cada acción realizada por todos los usuarios en todos los sitios web cubiertos por el servicio. -- `visits` es una tabla que contiene sesiones precompiladas en lugar de acciones individuales. - -Veamos y ejecutemos las consultas de tabla de creación real para estas tablas: - -``` sql -CREATE TABLE tutorial.hits_v1 -( - `WatchID` UInt64, - `JavaEnable` UInt8, - `Title` String, - `GoodEvent` Int16, - `EventTime` DateTime, - `EventDate` Date, - `CounterID` UInt32, - `ClientIP` UInt32, - `ClientIP6` FixedString(16), - `RegionID` UInt32, - `UserID` UInt64, - `CounterClass` Int8, - `OS` UInt8, - `UserAgent` UInt8, - `URL` String, - `Referer` String, - `URLDomain` String, - `RefererDomain` String, - `Refresh` UInt8, - `IsRobot` UInt8, - `RefererCategories` Array(UInt16), - `URLCategories` Array(UInt16), - `URLRegions` Array(UInt32), - `RefererRegions` Array(UInt32), - `ResolutionWidth` UInt16, - `ResolutionHeight` UInt16, - `ResolutionDepth` UInt8, - `FlashMajor` UInt8, - `FlashMinor` UInt8, - `FlashMinor2` String, - `NetMajor` UInt8, - `NetMinor` UInt8, - `UserAgentMajor` UInt16, - `UserAgentMinor` FixedString(2), - `CookieEnable` UInt8, - `JavascriptEnable` UInt8, - `IsMobile` UInt8, - `MobilePhone` UInt8, - `MobilePhoneModel` String, - `Params` String, - `IPNetworkID` UInt32, - `TraficSourceID` Int8, - `SearchEngineID` UInt16, - `SearchPhrase` String, - `AdvEngineID` UInt8, - `IsArtifical` UInt8, - `WindowClientWidth` UInt16, - `WindowClientHeight` UInt16, - `ClientTimeZone` Int16, - `ClientEventTime` DateTime, - `SilverlightVersion1` UInt8, - `SilverlightVersion2` UInt8, - `SilverlightVersion3` UInt32, - `SilverlightVersion4` UInt16, - `PageCharset` String, - `CodeVersion` UInt32, - `IsLink` UInt8, - `IsDownload` UInt8, - `IsNotBounce` UInt8, - `FUniqID` UInt64, - `HID` UInt32, - `IsOldCounter` UInt8, - `IsEvent` UInt8, - `IsParameter` UInt8, - `DontCountHits` UInt8, - `WithHash` UInt8, - `HitColor` FixedString(1), - `UTCEventTime` DateTime, - `Age` UInt8, - `Sex` UInt8, - `Income` UInt8, - `Interests` UInt16, - `Robotness` UInt8, - `GeneralInterests` Array(UInt16), - `RemoteIP` UInt32, - `RemoteIP6` FixedString(16), - `WindowName` Int32, - `OpenerName` Int32, - `HistoryLength` Int16, - `BrowserLanguage` FixedString(2), - `BrowserCountry` FixedString(2), - `SocialNetwork` String, - `SocialAction` String, - `HTTPError` UInt16, - `SendTiming` Int32, - `DNSTiming` Int32, - `ConnectTiming` Int32, - `ResponseStartTiming` Int32, - `ResponseEndTiming` Int32, - `FetchTiming` Int32, - `RedirectTiming` Int32, - `DOMInteractiveTiming` Int32, - `DOMContentLoadedTiming` Int32, - `DOMCompleteTiming` Int32, - `LoadEventStartTiming` Int32, - `LoadEventEndTiming` Int32, - `NSToDOMContentLoadedTiming` Int32, - `FirstPaintTiming` Int32, - `RedirectCount` Int8, - `SocialSourceNetworkID` UInt8, - `SocialSourcePage` String, - `ParamPrice` Int64, - `ParamOrderID` String, - `ParamCurrency` FixedString(3), - `ParamCurrencyID` UInt16, - `GoalsReached` Array(UInt32), - `OpenstatServiceName` String, - `OpenstatCampaignID` String, - `OpenstatAdID` String, - `OpenstatSourceID` String, - `UTMSource` String, - `UTMMedium` String, - `UTMCampaign` String, - `UTMContent` String, - `UTMTerm` String, - `FromTag` String, - `HasGCLID` UInt8, - `RefererHash` UInt64, - `URLHash` UInt64, - `CLID` UInt32, - `YCLID` UInt64, - `ShareService` String, - `ShareURL` String, - `ShareTitle` String, - `ParsedParams` Nested( - Key1 String, - Key2 String, - Key3 String, - Key4 String, - Key5 String, - ValueDouble Float64), - `IslandID` FixedString(16), - `RequestNum` UInt32, - `RequestTry` UInt8 -) -ENGINE = MergeTree() -PARTITION BY toYYYYMM(EventDate) -ORDER BY (CounterID, EventDate, intHash32(UserID)) -SAMPLE BY intHash32(UserID) -``` - -``` sql -CREATE TABLE tutorial.visits_v1 -( - `CounterID` UInt32, - `StartDate` Date, - `Sign` Int8, - `IsNew` UInt8, - `VisitID` UInt64, - `UserID` UInt64, - `StartTime` DateTime, - `Duration` UInt32, - `UTCStartTime` DateTime, - `PageViews` Int32, - `Hits` Int32, - `IsBounce` UInt8, - `Referer` String, - `StartURL` String, - `RefererDomain` String, - `StartURLDomain` String, - `EndURL` String, - `LinkURL` String, - `IsDownload` UInt8, - `TraficSourceID` Int8, - `SearchEngineID` UInt16, - `SearchPhrase` String, - `AdvEngineID` UInt8, - `PlaceID` Int32, - `RefererCategories` Array(UInt16), - `URLCategories` Array(UInt16), - `URLRegions` Array(UInt32), - `RefererRegions` Array(UInt32), - `IsYandex` UInt8, - `GoalReachesDepth` Int32, - `GoalReachesURL` Int32, - `GoalReachesAny` Int32, - `SocialSourceNetworkID` UInt8, - `SocialSourcePage` String, - `MobilePhoneModel` String, - `ClientEventTime` DateTime, - `RegionID` UInt32, - `ClientIP` UInt32, - `ClientIP6` FixedString(16), - `RemoteIP` UInt32, - `RemoteIP6` FixedString(16), - `IPNetworkID` UInt32, - `SilverlightVersion3` UInt32, - `CodeVersion` UInt32, - `ResolutionWidth` UInt16, - `ResolutionHeight` UInt16, - `UserAgentMajor` UInt16, - `UserAgentMinor` UInt16, - `WindowClientWidth` UInt16, - `WindowClientHeight` UInt16, - `SilverlightVersion2` UInt8, - `SilverlightVersion4` UInt16, - `FlashVersion3` UInt16, - `FlashVersion4` UInt16, - `ClientTimeZone` Int16, - `OS` UInt8, - `UserAgent` UInt8, - `ResolutionDepth` UInt8, - `FlashMajor` UInt8, - `FlashMinor` UInt8, - `NetMajor` UInt8, - `NetMinor` UInt8, - `MobilePhone` UInt8, - `SilverlightVersion1` UInt8, - `Age` UInt8, - `Sex` UInt8, - `Income` UInt8, - `JavaEnable` UInt8, - `CookieEnable` UInt8, - `JavascriptEnable` UInt8, - `IsMobile` UInt8, - `BrowserLanguage` UInt16, - `BrowserCountry` UInt16, - `Interests` UInt16, - `Robotness` UInt8, - `GeneralInterests` Array(UInt16), - `Params` Array(String), - `Goals` Nested( - ID UInt32, - Serial UInt32, - EventTime DateTime, - Price Int64, - OrderID String, - CurrencyID UInt32), - `WatchIDs` Array(UInt64), - `ParamSumPrice` Int64, - `ParamCurrency` FixedString(3), - `ParamCurrencyID` UInt16, - `ClickLogID` UInt64, - `ClickEventID` Int32, - `ClickGoodEvent` Int32, - `ClickEventTime` DateTime, - `ClickPriorityID` Int32, - `ClickPhraseID` Int32, - `ClickPageID` Int32, - `ClickPlaceID` Int32, - `ClickTypeID` Int32, - `ClickResourceID` Int32, - `ClickCost` UInt32, - `ClickClientIP` UInt32, - `ClickDomainID` UInt32, - `ClickURL` String, - `ClickAttempt` UInt8, - `ClickOrderID` UInt32, - `ClickBannerID` UInt32, - `ClickMarketCategoryID` UInt32, - `ClickMarketPP` UInt32, - `ClickMarketCategoryName` String, - `ClickMarketPPName` String, - `ClickAWAPSCampaignName` String, - `ClickPageName` String, - `ClickTargetType` UInt16, - `ClickTargetPhraseID` UInt64, - `ClickContextType` UInt8, - `ClickSelectType` Int8, - `ClickOptions` String, - `ClickGroupBannerID` Int32, - `OpenstatServiceName` String, - `OpenstatCampaignID` String, - `OpenstatAdID` String, - `OpenstatSourceID` String, - `UTMSource` String, - `UTMMedium` String, - `UTMCampaign` String, - `UTMContent` String, - `UTMTerm` String, - `FromTag` String, - `HasGCLID` UInt8, - `FirstVisit` DateTime, - `PredLastVisit` Date, - `LastVisit` Date, - `TotalVisits` UInt32, - `TraficSource` Nested( - ID Int8, - SearchEngineID UInt16, - AdvEngineID UInt8, - PlaceID UInt16, - SocialSourceNetworkID UInt8, - Domain String, - SearchPhrase String, - SocialSourcePage String), - `Attendance` FixedString(16), - `CLID` UInt32, - `YCLID` UInt64, - `NormalizedRefererHash` UInt64, - `SearchPhraseHash` UInt64, - `RefererDomainHash` UInt64, - `NormalizedStartURLHash` UInt64, - `StartURLDomainHash` UInt64, - `NormalizedEndURLHash` UInt64, - `TopLevelDomain` UInt64, - `URLScheme` UInt64, - `OpenstatServiceNameHash` UInt64, - `OpenstatCampaignIDHash` UInt64, - `OpenstatAdIDHash` UInt64, - `OpenstatSourceIDHash` UInt64, - `UTMSourceHash` UInt64, - `UTMMediumHash` UInt64, - `UTMCampaignHash` UInt64, - `UTMContentHash` UInt64, - `UTMTermHash` UInt64, - `FromHash` UInt64, - `WebVisorEnabled` UInt8, - `WebVisorActivity` UInt32, - `ParsedParams` Nested( - Key1 String, - Key2 String, - Key3 String, - Key4 String, - Key5 String, - ValueDouble Float64), - `Market` Nested( - Type UInt8, - GoalID UInt32, - OrderID String, - OrderPrice Int64, - PP UInt32, - DirectPlaceID UInt32, - DirectOrderID UInt32, - DirectBannerID UInt32, - GoodID String, - GoodName String, - GoodQuantity Int32, - GoodPrice Int64), - `IslandID` FixedString(16) -) -ENGINE = CollapsingMergeTree(Sign) -PARTITION BY toYYYYMM(StartDate) -ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) -SAMPLE BY intHash32(UserID) -``` - -Puede ejecutar esas consultas utilizando el modo interactivo de `clickhouse-client` (simplemente ejecútelo en un terminal sin especificar una consulta por adelantado) o pruebe algunos [interfaz alternativa](../interfaces/index.md) Si quieres. - -Como podemos ver, `hits_v1` utiliza el [motor básico MergeTree](../engines/table-engines/mergetree-family/mergetree.md), mientras que el `visits_v1` utiliza el [Derrumbar](../engines/table-engines/mergetree-family/collapsingmergetree.md) variante. - -### Importar datos {#import-data} - -La importación de datos a ClickHouse se realiza a través de [INSERT INTO](../sql-reference/statements/insert-into.md) consulta como en muchas otras bases de datos SQL. Sin embargo, los datos generalmente se proporcionan en uno de los [Formatos de serialización compatibles](../interfaces/formats.md) en lugar de `VALUES` cláusula (que también es compatible). - -Los archivos que descargamos anteriormente están en formato separado por tabuladores, así que aquí le mostramos cómo importarlos a través del cliente de la consola: - -``` bash -clickhouse-client --query "INSERT INTO tutorial.hits_v1 FORMAT TSV" --max_insert_block_size=100000 < hits_v1.tsv -clickhouse-client --query "INSERT INTO tutorial.visits_v1 FORMAT TSV" --max_insert_block_size=100000 < visits_v1.tsv -``` - -ClickHouse tiene un montón de [ajustes para sintonizar](../operations/settings/index.md) y una forma de especificarlos en el cliente de la consola es a través de argumentos, como podemos ver con `--max_insert_block_size`. La forma más fácil de averiguar qué configuraciones están disponibles, qué significan y cuáles son los valores predeterminados es consultar el `system.settings` tabla: - -``` sql -SELECT name, value, changed, description -FROM system.settings -WHERE name LIKE '%max_insert_b%' -FORMAT TSV - -max_insert_block_size 1048576 0 "The maximum block size for insertion, if we control the creation of blocks for insertion." -``` - -Opcionalmente se puede [OPTIMIZE](../sql-reference/statements/misc.md#misc_operations-optimize) las tablas después de la importación. Las tablas que están configuradas con un motor de la familia MergeTree siempre fusionan partes de datos en segundo plano para optimizar el almacenamiento de datos (o al menos verificar si tiene sentido). Estas consultas obligan al motor de tablas a realizar la optimización del almacenamiento en este momento en lugar de algún tiempo después: - -``` bash -clickhouse-client --query "OPTIMIZE TABLE tutorial.hits_v1 FINAL" -clickhouse-client --query "OPTIMIZE TABLE tutorial.visits_v1 FINAL" -``` - -Estas consultas inician una operación intensiva de E / S y CPU, por lo que si la tabla recibe datos nuevos de manera consistente, es mejor dejarlos solos y dejar que las fusiones se ejecuten en segundo plano. - -Ahora podemos comprobar si la importación de la tabla fue exitosa: - -``` bash -clickhouse-client --query "SELECT COUNT(*) FROM tutorial.hits_v1" -clickhouse-client --query "SELECT COUNT(*) FROM tutorial.visits_v1" -``` - -## Consultas de ejemplo {#example-queries} - -``` sql -SELECT - StartURL AS URL, - AVG(Duration) AS AvgDuration -FROM tutorial.visits_v1 -WHERE StartDate BETWEEN '2014-03-23' AND '2014-03-30' -GROUP BY URL -ORDER BY AvgDuration DESC -LIMIT 10 -``` - -``` sql -SELECT - sum(Sign) AS visits, - sumIf(Sign, has(Goals.ID, 1105530)) AS goal_visits, - (100. * goal_visits) / visits AS goal_percent -FROM tutorial.visits_v1 -WHERE (CounterID = 912887) AND (toYYYYMM(StartDate) = 201403) AND (domain(StartURL) = 'yandex.ru') -``` - -## Implementación de clúster {#cluster-deployment} - -El clúster ClickHouse es un clúster homogéneo. Pasos para configurar: - -1. Instale el servidor ClickHouse en todas las máquinas del clúster -2. Configurar configuraciones de clúster en archivos de configuración -3. Crear tablas locales en cada instancia -4. Crear un [Tabla distribuida](../engines/table-engines/special/distributed.md) - -[Tabla distribuida](../engines/table-engines/special/distributed.md) es en realidad una especie de “view” a las tablas locales del clúster ClickHouse. La consulta SELECT de una tabla distribuida se ejecuta utilizando recursos de todos los fragmentos del clúster. Puede especificar configuraciones para varios clústeres y crear varias tablas distribuidas que proporcionen vistas a diferentes clústeres. - -Ejemplo de configuración para un clúster con tres fragmentos, una réplica cada uno: - -``` xml - - - - - example-perftest01j.yandex.ru - 9000 - - - - - example-perftest02j.yandex.ru - 9000 - - - - - example-perftest03j.yandex.ru - 9000 - - - - -``` - -Para más demostraciones, vamos a crear una nueva tabla local con la misma `CREATE TABLE` consulta que utilizamos para `hits_v1`, pero nombre de tabla diferente: - -``` sql -CREATE TABLE tutorial.hits_local (...) ENGINE = MergeTree() ... -``` - -Creación de una tabla distribuida que proporcione una vista en las tablas locales del clúster: - -``` sql -CREATE TABLE tutorial.hits_all AS tutorial.hits_local -ENGINE = Distributed(perftest_3shards_1replicas, tutorial, hits_local, rand()); -``` - -Una práctica común es crear tablas distribuidas similares en todas las máquinas del clúster. Permite ejecutar consultas distribuidas en cualquier máquina del clúster. También hay una opción alternativa para crear una tabla distribuida temporal para una consulta SELECT determinada usando [remoto](../sql-reference/table-functions/remote.md) función de la tabla. - -Vamos a correr [INSERT SELECT](../sql-reference/statements/insert-into.md) en la tabla Distributed para extender la tabla a varios servidores. - -``` sql -INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1; -``` - -!!! warning "Aviso" - Este enfoque no es adecuado para la fragmentación de tablas grandes. Hay una herramienta separada [Método de codificación de datos:](../operations/utilities/clickhouse-copier.md) que puede volver a fragmentar tablas grandes arbitrarias. - -Como era de esperar, las consultas computacionalmente pesadas se ejecutan N veces más rápido si utilizan 3 servidores en lugar de uno. - -En este caso, hemos utilizado un clúster con 3 fragmentos, y cada uno contiene una sola réplica. - -Para proporcionar resiliencia en un entorno de producción, se recomienda que cada fragmento contenga 2-3 réplicas distribuidas entre varias zonas de disponibilidad o centros de datos (o al menos racks). Tenga en cuenta que ClickHouse admite un número ilimitado de réplicas. - -Ejemplo de configuración para un clúster de un fragmento que contiene tres réplicas: - -``` xml - - ... - - - - example-perftest01j.yandex.ru - 9000 - - - example-perftest02j.yandex.ru - 9000 - - - example-perftest03j.yandex.ru - 9000 - - - - -``` - -Para habilitar la replicación nativa [ZooKeeper](http://zookeeper.apache.org/) se requiere. ClickHouse se encarga de la coherencia de los datos en todas las réplicas y ejecuta el procedimiento de restauración después de la falla automáticamente. Se recomienda implementar el clúster ZooKeeper en servidores independientes (donde no se están ejecutando otros procesos, incluido ClickHouse). - -!!! note "Nota" - ZooKeeper no es un requisito estricto: en algunos casos simples, puede duplicar los datos escribiéndolos en todas las réplicas de su código de aplicación. Este enfoque es **ni** recomendado, en este caso, ClickHouse no podrá garantizar la coherencia de los datos en todas las réplicas. Por lo tanto, se convierte en responsabilidad de su aplicación. - -Las ubicaciones de ZooKeeper se especifican en el archivo de configuración: - -``` xml - - - zoo01.yandex.ru - 2181 - - - zoo02.yandex.ru - 2181 - - - zoo03.yandex.ru - 2181 - - -``` - -Además, necesitamos establecer macros para identificar cada fragmento y réplica que se utilizan en la creación de tablas: - -``` xml - - 01 - 01 - -``` - -Si no hay réplicas en este momento en la creación de la tabla replicada, se crea una instancia de una nueva primera réplica. Si ya hay réplicas activas, la nueva réplica clona los datos de las existentes. Tiene la opción de crear primero todas las tablas replicadas y, a continuación, insertar datos en ella. Otra opción es crear algunas réplicas y agregar las otras después o durante la inserción de datos. - -``` sql -CREATE TABLE tutorial.hits_replica (...) -ENGINE = ReplcatedMergeTree( - '/clickhouse_perftest/tables/{shard}/hits', - '{replica}' -) -... -``` - -Aquí usamos [ReplicatedMergeTree](../engines/table-engines/mergetree-family/replication.md) motor de mesa. En los parámetros, especificamos la ruta ZooKeeper que contiene identificadores de fragmentos y réplicas. - -``` sql -INSERT INTO tutorial.hits_replica SELECT * FROM tutorial.hits_local; -``` - -La replicación funciona en modo multi-master. Los datos se pueden cargar en cualquier réplica y el sistema los sincroniza automáticamente con otras instancias. La replicación es asíncrona, por lo que en un momento dado, no todas las réplicas pueden contener datos insertados recientemente. Al menos una réplica debe estar disponible para permitir la ingestión de datos. Otros sincronizarán los datos y repararán la coherencia una vez que vuelvan a activarse. Tenga en cuenta que este enfoque permite la baja posibilidad de una pérdida de datos recientemente insertados. - -[Artículo Original](https://clickhouse.tech/docs/en/getting_started/tutorial/) diff --git a/docs/es/guides/apply-catboost-model.md b/docs/es/guides/apply-catboost-model.md deleted file mode 100644 index b1fe50f3276..00000000000 --- a/docs/es/guides/apply-catboost-model.md +++ /dev/null @@ -1,239 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: "Aplicaci\xF3n de modelos CatBoost" ---- - -# Aplicación de un modelo Catboost en ClickHouse {#applying-catboost-model-in-clickhouse} - -[CatBoost](https://catboost.ai) es una biblioteca de impulso de gradiente libre y de código abierto desarrollada en [Yandex](https://yandex.com/company/) para el aprendizaje automático. - -Con esta instrucción, aprenderá a aplicar modelos preentrenados en ClickHouse ejecutando la inferencia de modelos desde SQL. - -Para aplicar un modelo CatBoost en ClickHouse: - -1. [Crear una tabla](#create-table). -2. [Insertar los datos en la tabla](#insert-data-to-table). -3. [Integrar CatBoost en ClickHouse](#integrate-catboost-into-clickhouse) (Paso opcional). -4. [Ejecute la inferencia del modelo desde SQL](#run-model-inference). - -Para obtener más información sobre la formación de modelos CatBoost, consulte [Entrenamiento y aplicación de modelos](https://catboost.ai/docs/features/training.html#training). - -## Requisito {#prerequisites} - -Si no tienes el [Acoplador](https://docs.docker.com/install/) sin embargo, instalarlo. - -!!! note "Nota" - [Acoplador](https://www.docker.com) es una plataforma de software que le permite crear contenedores que aíslan una instalación de CatBoost y ClickHouse del resto del sistema. - -Antes de aplicar un modelo CatBoost: - -**1.** Tire de la [Imagen de acoplador](https://hub.docker.com/r/yandex/tutorial-catboost-clickhouse) del registro: - -``` bash -$ docker pull yandex/tutorial-catboost-clickhouse -``` - -Esta imagen de Docker contiene todo lo que necesita para ejecutar CatBoost y ClickHouse: código, tiempo de ejecución, bibliotecas, variables de entorno y archivos de configuración. - -**2.** Asegúrese de que la imagen de Docker se haya extraído correctamente: - -``` bash -$ docker image ls -REPOSITORY TAG IMAGE ID CREATED SIZE -yandex/tutorial-catboost-clickhouse latest 622e4d17945b 22 hours ago 1.37GB -``` - -**3.** Inicie un contenedor Docker basado en esta imagen: - -``` bash -$ docker run -it -p 8888:8888 yandex/tutorial-catboost-clickhouse -``` - -## 1. Crear una tabla {#create-table} - -Para crear una tabla ClickHouse para el ejemplo de capacitación: - -**1.** Inicie el cliente de consola ClickHouse en el modo interactivo: - -``` bash -$ clickhouse client -``` - -!!! note "Nota" - El servidor ClickHouse ya se está ejecutando dentro del contenedor Docker. - -**2.** Cree la tabla usando el comando: - -``` sql -:) CREATE TABLE amazon_train -( - date Date MATERIALIZED today(), - ACTION UInt8, - RESOURCE UInt32, - MGR_ID UInt32, - ROLE_ROLLUP_1 UInt32, - ROLE_ROLLUP_2 UInt32, - ROLE_DEPTNAME UInt32, - ROLE_TITLE UInt32, - ROLE_FAMILY_DESC UInt32, - ROLE_FAMILY UInt32, - ROLE_CODE UInt32 -) -ENGINE = MergeTree ORDER BY date -``` - -**3.** Salir del cliente de la consola ClickHouse: - -``` sql -:) exit -``` - -## 2. Insertar los datos en la tabla {#insert-data-to-table} - -Para insertar los datos: - -**1.** Ejecute el siguiente comando: - -``` bash -$ clickhouse client --host 127.0.0.1 --query 'INSERT INTO amazon_train FORMAT CSVWithNames' < ~/amazon/train.csv -``` - -**2.** Inicie el cliente de consola ClickHouse en el modo interactivo: - -``` bash -$ clickhouse client -``` - -**3.** Asegúrese de que los datos se hayan cargado: - -``` sql -:) SELECT count() FROM amazon_train - -SELECT count() -FROM amazon_train - -+-count()-+ -| 65538 | -+-------+ -``` - -## 3. Integrar CatBoost en ClickHouse {#integrate-catboost-into-clickhouse} - -!!! note "Nota" - **Paso opcional.** La imagen de Docker contiene todo lo que necesita para ejecutar CatBoost y ClickHouse. - -Para integrar CatBoost en ClickHouse: - -**1.** Construir la biblioteca de evaluación. - -La forma más rápida de evaluar un modelo CatBoost es compilar `libcatboostmodel.` biblioteca. Para obtener más información acerca de cómo construir la biblioteca, vea [Documentación de CatBoost](https://catboost.ai/docs/concepts/c-plus-plus-api_dynamic-c-pluplus-wrapper.html). - -**2.** Cree un nuevo directorio en cualquier lugar y con cualquier nombre, por ejemplo, `data` y poner la biblioteca creada en ella. La imagen de Docker ya contiene la biblioteca `data/libcatboostmodel.so`. - -**3.** Cree un nuevo directorio para el modelo de configuración en cualquier lugar y con cualquier nombre, por ejemplo, `models`. - -**4.** Cree un archivo de configuración de modelo con cualquier nombre, por ejemplo, `models/amazon_model.xml`. - -**5.** Describir la configuración del modelo: - -``` xml - - - - catboost - - amazon - - /home/catboost/tutorial/catboost_model.bin - - 0 - - -``` - -**6.** Agregue la ruta de acceso a CatBoost y la configuración del modelo a la configuración de ClickHouse: - -``` xml - -/home/catboost/data/libcatboostmodel.so -/home/catboost/models/*_model.xml -``` - -## 4. Ejecute la inferencia del modelo desde SQL {#run-model-inference} - -Para el modelo de prueba, ejecute el cliente ClickHouse `$ clickhouse client`. - -Asegurémonos de que el modelo esté funcionando: - -``` sql -:) SELECT - modelEvaluate('amazon', - RESOURCE, - MGR_ID, - ROLE_ROLLUP_1, - ROLE_ROLLUP_2, - ROLE_DEPTNAME, - ROLE_TITLE, - ROLE_FAMILY_DESC, - ROLE_FAMILY, - ROLE_CODE) > 0 AS prediction, - ACTION AS target -FROM amazon_train -LIMIT 10 -``` - -!!! note "Nota" - Función [modelEvaluar](../sql-reference/functions/other-functions.md#function-modelevaluate) devuelve tupla con predicciones sin procesar por clase para modelos multiclase. - -Vamos a predecir la probabilidad: - -``` sql -:) SELECT - modelEvaluate('amazon', - RESOURCE, - MGR_ID, - ROLE_ROLLUP_1, - ROLE_ROLLUP_2, - ROLE_DEPTNAME, - ROLE_TITLE, - ROLE_FAMILY_DESC, - ROLE_FAMILY, - ROLE_CODE) AS prediction, - 1. / (1 + exp(-prediction)) AS probability, - ACTION AS target -FROM amazon_train -LIMIT 10 -``` - -!!! note "Nota" - Más información sobre [exp()](../sql-reference/functions/math-functions.md) función. - -Vamos a calcular LogLoss en la muestra: - -``` sql -:) SELECT -avg(tg * log(prob) + (1 - tg) * log(1 - prob)) AS logloss -FROM -( - SELECT - modelEvaluate('amazon', - RESOURCE, - MGR_ID, - ROLE_ROLLUP_1, - ROLE_ROLLUP_2, - ROLE_DEPTNAME, - ROLE_TITLE, - ROLE_FAMILY_DESC, - ROLE_FAMILY, - ROLE_CODE) AS prediction, - 1. / (1. + exp(-prediction)) AS prob, - ACTION AS tg - FROM amazon_train -) -``` - -!!! note "Nota" - Más información sobre [avg()](../sql-reference/aggregate-functions/reference.md#agg_function-avg) y [registro()](../sql-reference/functions/math-functions.md) función. - -[Artículo Original](https://clickhouse.tech/docs/en/guides/apply_catboost_model/) diff --git a/docs/es/guides/index.md b/docs/es/guides/index.md deleted file mode 100644 index c8332ac7846..00000000000 --- a/docs/es/guides/index.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Guiar -toc_priority: 38 -toc_title: "Descripci\xF3n" ---- - -# Guías de ClickHouse {#clickhouse-guides} - -Lista de instrucciones detalladas paso a paso que ayudan a resolver varias tareas usando ClickHouse: - -- [Tutorial sobre la configuración simple del clúster](../getting-started/tutorial.md) -- [Aplicación de un modelo CatBoost en ClickHouse](apply-catboost-model.md) - -[Artículo Original](https://clickhouse.tech/docs/en/guides/) diff --git a/docs/es/images/column-oriented.gif b/docs/es/images/column-oriented.gif deleted file mode 100644 index d5ac7c82848cb633cb0f1b341909c0b955e79fcf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 43771 zcmd42RajilwlCUP&_GCn6Ck)lNN^AC8XOYb-M#T{XxtiicXv&2_r?kC5?uS{zxH1H z+=sjFcOK5enJ=?y%%N4I>YG*dD=8UC9$rH?L_~xe1OR*u2>%U)4*}pE0C+C|-VcDE zcf&8c;k!WiC;)z(4PO+24+G)TK=@%dd=Usw0l@PB@LB-83;OZv%_!a;VY`}^Gf*okB7sO?d`3DgM;hq zOCK+f;-Z4|w3M~ARb6e(nVD%7C3!OwqmGXDxY+2y!2w5m+q&B7u+X5Fot^vZW4Md+ z(}vyS-Jh$Im6zwomyQnjMH8Hi41RfeeYOGD)Pz4B-Tv8!M3YwYwPm9?f1D;=W5pU?&{g~pZllD-L1RhR`|Xq{4@}Lkp%zr=_NV& zWr`R+js-u6j%|-?`ej$#c(Ey#=It!!a@`R9(*3Q4z{K)h7=yQHg?W@9)gsHCiX_Ae1FUTkl83H z{>9>AEl4T)&j*TsY>L@CnNo1Fa+Z=H>uhmC`sg`Jm= z^D`ei2gQFslz*)`nV9jZic9{tt-m!vN(&bk2R=47cXxMI_s^{MPUdW%czJo**g4oZ zI9UF2usD0#xfpt|*f~@ETZ6c%v$2z*gE`=T03Xf ze}%(pY;VivVd%j2iIx4Ikp9i6pz#0CYHRyH+|DklrvJ;||4+rv>Yfg!Y^tWt_O4FG zrhnO~{xRjiC+1{o=wk1rZf|e%?^aZ{uy?U{wy<}g5EJ`Xt5MJ^7#dsJ{X=BMzhzDSm#pZ2l>H|mY#shu7B_XWbTc)Pbh5Xl_*dY3mjBro?*FLo zzhzDSvoAdVQI_qmGi?8K@Bgd2|4sUvL;n>2Cwc!?{wMuS?f&Mx)8FJqfWJIHJwDvu z-QHYZUH-W^KRZ1+J~}+u-`m~U-rD@VvA(vtvb?kiTbQ4lotd7RoERS)9T^@P9O&=+ z^|QC9yQ{OKy{)ySxv8&_?ryFw&Q6XF_I9>5)>f7l=4PfQ#zuxez8mQ4 z>FQ`}X=;2^S5s9{R#H@umy?x|mXeeZ7yBwIA}k~*z|Y6a!_CF{9G$T-PN(EYVG4a2zBMQXEvHqTE6*`jwcs78$JI zvtqFu91yIFL%kU27f4S(gi9zIl^*Q)d37{W6i9l3{u9JQCM(EQi&|$eX-^dZRUZ(O zk)aZVhvpx*{*oH0llAF-?U`7rT5W`cN#+Rx@x*?KeTpcf9(q&F5#fNhy_Jhg863*< ziZZzH4UKg0_7ws$*o07g_xcS-)u*`jwBgWSq}!;t)Pf+w7Se?aIwo9i#(q}|&)-rb@(~`2y@0C@HiA2fdsAxnDos!rO zOau*jm6N#+sBe+X-^5B5Ii3&Z1^FyVF16`3pR2^+sBQlCk6+ zzLB_V{UHOB86pOvxa)xEU}k#odq-qZDFvPHcKr&--=(>Tf1A{ADq#Nyn@@oDR1X7rt$?9!LJzF zxAAamc+x*i<+tq7K#GU2xFScf`QO*Ze4`zI=k!KkTm91KAb9%^?`KEBU&@OvkyEhi zAN8j`w+4u=A00g}VbZsM09<~CK~ha|#3a-BcIp?O)dt)(lyocl zpttRN@BKnVrdhExO=|RQl z(`L0^@3U_Ihey7jsAtj_{dDAne@eS5pQuNL*`7tGG}WH3g7l)FZ{{ts+Adcd_LUH)s zqWe{8U_dW0dxv_z8zU+(u&bM*ug5X?!tdqZc<+ml5?+9$X9KWf2F0nDx?d?ka^I)i{bJCW z0)1=W#H)je(&oN3`t8+0uUl0`P8`ggEBg`aY^me)Sri><(+Ebv6)ZX z)M5<7Dt5IA83LvZGA**-a^vDEN2!@4ct@1xIbxf(O;}BdqiFQml;aA3>QF3B z_aF^Yy~@S!pqHd7oeDLp&%=JEmu6s`j*UymqrIk=RnSpQs{B%b=UXDF%cq(eI9rI( zXDQ{%raIdpVt2PPklY?dP-=czr2v!{TfO z*|&79$Rhg4)NlD-bE!J6j9MdB^Ob&TrRuh&x;Vocz|MwBw>~<~z6Y7w+=dFB3~Rla z!}+q*Zxyl%HU>Xi7y2p_GyVIlc}O0$6lvcViuT!ucbYHaL^(0%d{mnXb*oQ3 zv9S!XT8fNIDk^<#X|3-)vnVv%i22P{B{FQbm$I=oD#OMB4^yidTeErK1O0cf_rf3_ zcj?F+%~wzBm99ZkyYAPa=WyGPqd$;}=6i62 zPC~vfFJVwMcbA%0gWk0-Z;UOp-M_B(Qo~w*e9`Qlwyg>`K3>0Da_=7w`2>6%Hn<;d z?i=g72>12bB*97Pl3x215xw%AVj0>mgY887d}u<`1?%H8b4r}jHD>W98rFK@NOmUO zVdf?pwP>Uc5QY4nvxANm`_=^u9xtLa5h1wrUuFm&t71R4j5|d+D{0GF#3T`VNYK{C z^fB)+@p}y!hBD_9`R+={wN52Wo9EI>VMXXb#gy1B>r{tiW2NJXwPEto>Nf@M_8l*MSIwo_Iy>$} z`;H^OF7MSFp~jkk5B3J|whdiHu5Rv=BOh`BgdttvFFM7GaNYDx4CU@V?r}%3dHWUx zoM%w1@*-AOZ=1G@XIO3RBC)Q0hsOVY$oR!6c_Dq5>N&Pd3bJHfP)gs`k7uH1IH25am)~?t7$K<+YG}>Kf z)u1VGg88Ox#@f~)aDDj{W1y{j=e51Y?ehY{1b+>>+#TEvGjla|#k~@K>h4dqcH!m! z^b>d7eu&NI>^m#hI*GsMFcZIr-azv*Sjej(Oz5VK^+VfA* z^w`{Uh~bv+|LFr;@cvQcyGrMavF~FX=F@KO`+m@u*F0b%Ip8hSH$Ni@^wPXy$jvEcvMqZ)=8VO4|!d2I~<9fAdgywQono;t6xn3L}{eAlvu0E)J_%2!PSKIS_yfnu40+5gD=(LqVW0C_*ttX+F=o$cM%;hL|CG~cCj@1Bw{xd(sPF(PYC?l95Pr8nche2 zg@q^-2W#K>opXkkB?-*jMYd^0?rXpyU>C^7T?9)vWb+mxW)Yn6K5}nAVzW6)`OaTY z0w{DBydv%2Qy2bn7hM7iH>VH4v;YGy|;fO@x1eQ*(?qf@#G1V49n4W<|cQLe{ac@iFnk*vvHG_Deu_Pt2q{$K0 z@1m!9qUU)+(0@b;5k(zBqZg!uf6>dGYbDHr^dF%K>cx?X1c|30>9zfYeIAG=EO7^x zZ~=`Pqfa;iMV=xj385#5+=L8BBu1Dgnt_4OcX0rsIINU7D$O_-1|)jRct3D5FS-kB zOR^79JhvzGi={X7eR4Pv9Lm!Y|3L;?P7ls&hN>1NyDTPCfuR^JDU94uZ_8wec6<^; zd^{So_72KjoJuedUy_%odth)Ql_0^8bcGf=u9dVKoMeceSUc}$3yxmCOM6OAtR9N6 z@=TLskgZQilCt#Il1`HEOWT8{BMc_R>?Ykwrg!6~#1X|Lq=XfwI6?2@QngbvTH?0A zF4-w5p()UF-gvD2l#-Uz8||zZOM!LtcqDKtsb(gvXXepR$_qG4U?Fq&KJ!mWJf=>{ zzD#!HP^vGXPrqln5<^-dfy@|D(nLy*PH~#LN78_0269+NRAI*Jfs6+;0sQcEJ?-db zUKtXdj1Q@4nkDdrNuJzJ%RF{QPkJv02j1YdmaLG)th|(Dw8L0Lj2xt;{I|oIj|{0- zo~dfYIhXg@7c!|>R@oK2`6|Sj79J@w7_oAx1zI}U@0Q}emuBi6LSHjxDyA0dG8Wm( zI#z=DScs$8ta7WqbxIsPgQH|i4IVO7!wWT2odFNAraDDtIt4C=&c>zr7<@%a zURl`0MLNS}U9Dvb$)#~ErBX|U)ShLo_u%sIa&x}2nxb;tp~8mY{O*S^KP|5+i;ADb zB`ka;0*7#oWWLB0*%HWb)J#d9sAcrQP2Pe>Y1*(fUTNhPC~xZ_o>MmI!m2X*P&!Mt zGUqU)d$HtsshIJ>;g>A5U$(4*(YdNLf7GkIcDSO~%V}CCx#h61)vIDLHFHjd=Y_8mFkf6Z7F1D zIFG`svQ@kIN;W$0(BqWR=KL`D5+iLRwHU@(KcrRvaF~dkRu6xOMi2BboiR1)@h99zq+eNg7lgM}$tj|fLOMwD zUbov-$n~f^cE2HLYmdm0jOadX>kjnl&P&Vt(B5^!-?;DIQ^?r9D&4+L@^fRkeLfW4 znUB}Ok>1Ysqhn3d@XN8y1i#v9+Rs6f&V{s2(RQ^D^X-%PMuc2ORHk;#V%^?+)qT`m z`}Do!rM=kgZKwP_(#u`9pkB?Cp1of^-;aB~g8Lff`l&|yq|55*M(Z`0O_=riS>=03 zNCj{w2mGD}fTV-|D}Zn1>EXx#EC51y`ygax5Hbn?Z1n4<_qId;z{i77$PgBwKfkO; z?Y@hFxVxaAgpqth+j)tDng3Xw_H{3_Nu*Q1IM`!eU^wUO<%?GbN241g>=IO!xpS@C!1&R33dG;DM=MvL`B{nuvy)~jSGM#s%+bciBAU~5)I+vw4t>-g+{xoX9GTv=7u5mKChc!iI>!CL` zj`v*2qu)dHJk_VSp#3~gjx`fRGeZQKd*PqQ!J5x}CGh8XSn7GEr5!%|s0Ry=oJD$u zNv^^`6|;yPF!>BvlAwxm2A^ujG>z?y<;gTf$2iLJ`~}9+DCxpijB(G7sr<+#HkKu; zv4zseQDf3+Bdz7?v}xeUg6ZnC)`#UD!KIsp1-Fhdk4US)XU|~42}s6lO61~K{l&Ns zi(v|jvl)gnEUWP+vsX{HQ?J+B1Qw`lmfTmDe^xA@bB||!NC##tp9#POq~}T>m%pt} znk)2vQdp|AUK@N~tHEC%6U`wy zn2O&DAJ$fUXX~F<3H(-JzKb9HHs3ZZ7SAl6KQ0bsOdfP>Yzx3=tUH36KWw~x8Ea>m z85mnr^O(Pnoc|@b)o;6TtI$7EVOHMJ@FKYVIdcu8bKyyF^yPVhdwl+N=fIdPjPi6< z-)@rxdzBP>m;Q8T?!zYk>M!~9AIi+Tm<5|TKR2yLUyi9aOoeHnyiJ00_I29qVIQ7% z9{v$JyE#3*h&p4#x%>h@1!@_arWwf>gvoGR=1o8S3f^7${J z-_1csog-6(@P112ahJ@a-l+5Rmn%Nj`PXa*NJO3!K1r%v16N&5kp-un-dU9Wu@~MI_P=zGzCB*M<$yob z+uu5_A67T)y4pXs2w!5E9wkv-57-^NC%;b=ei(p5@F(uJkFV%)t~*wOlHp6=golQO zZzc@RvvDp^25+Xx=Vre@kguN)`CofS-K$mJPpn^0!LQ$U{`oX0UJI?qH(C z^83^9>C58r0oMA(E2XPk@;~pYp7AE0juBtpKDu)Jg!O?% zP(k1J!%194$JlDesu5|r2!v_vZfdjQ<=N1Vm{swVRTwK89Mf5OhIsmdZ4-<8|-*dQ?}qRuV% zi=no(!YAJLr@(`wnQ}&pmEkz_zwI|g1UcDKpy7A>Xl*S1h)5Qw0GGyAAo~#$esnG}EeiKGMyL){Fhh-6qA}FWOB{DU!D3hZDCEEW|61F!O!B>VgSdStsG$-rq66X8M1Bm5%N~5 z^-3bjsygI4Vs5E&^`@tVjUUA~x{ep&Ru5zFpVo*xc#;Z<=Q7#3oGvw)w9OJ@(R5&o znOLbNe1(-#<8ddJ6k@89mT3`;3_AuCSgm}DXu#LvC}Z*)bt*4SO5d!&nGlHVFsM9A zYdHckRkS^0^Fzkta95n$CB-mZdPIB6svEutKSRbk9@4>`UVqA5X4D=>rn~j5^}oQ* zLRq-dFw;)H%vv!~an`6C8(j+tWMy1Wxwunbjf=03IZPX0u5!-=);$g`g)+UX31kQw zUD(u(OyB6Q4m@PsNemk$*d|h3xf{xecwpR*pMWzGj@AtO@XXr!D{wJ5%7wZf)hgqd>8k_{7hu^rCH`g#}e3j-=nxd4qB5R7ope^ol>qE!4KS} zOcstvv2&rv)JG!PuR%GT(70C1xIGQ|Y@b-@GxkVyBv9PBFLC5Ki?ntnYtj2&qbew* zjAAUDP7qJS)|X8?^HB?}_tG{vbnTu3-M+b(;!ru14r}d<@OLkP!MWKdaWb4T6EV`< zbCB@+zIWE!VpNEqLM})Q@9NPcrKugwZq?wjh~HlIbY0myr+fVXOcZxb{N4cqHq-YM zkwthp6r+9Q6OmP1T0aS4g&N3{GxjH^(Gzoog{@mx-HB2(F{10)xIN(*onK4uLH8;1RvUx5=d4Ki++q0^&e_ zKz}h8eu}t9P=0zFADOTt`Qu}R;$*K!+20L#{gj4#mFT2Pue=>r+HQ@8Q72qVbKXzx z1L%dH0*{dMksgZ`-TL1}n~HwmCU4+u;*1Z6zg?jGj6knT&$X)7mKB_uN8S)gCcNk?)C=xF?R{Fi4Gjdp8n`Fyqz5`Zwv73+1 z`tU=LoQbB>uc0~+h1q(Q7*^goU+rjXYb(>D6|}~gR~#8Osd9itX*~+egQq)5zZi&1}hnTd85Ew z58=ZM8*izj#fNEv&P$e3zvq#4@<+{*+ts|dNIg8jS@S1GvST+#pB-H5lxm_v$3SO! z-2e-Zg12rL2@PsH1VP$8TxOIp^P{`|2HO41q@Tk^Moq)gwYx1kYEtX4_5{A)b$?5A z&OU%FbKT-i@U1!9daZ0zz%ds)V_BUF!mvzbs8Xhr#_JKmJ|;6pOG6s5wR!$=yEt`P z1zIRyAh@Jv-%6K80vlOCdp1T=8B3o^{H{yumJdbXM3W#lvzn?B3#8A)^Pa1mRgVJS zwBfw49yiW9>&TtTX=1-2v-4W}4m5NASJNV74TXZw+j8y?EBW8JTjQTjUxwROb|tq8 zPkeP8KNl`E;=4iGPWIT$A7NHB9#RM|Q+f;P>tBDnmq(E3SrLpZZFIUdve=y%EfX!T zud)6l!9M9UBB>SdvO+p7MV%!0sPe&_9u#oA>S zlHmS1D_;@bJLhbkC-^2yRr5^KlS|S&{vEaguUsSpqov=}2gTSe-FZ=t)oP@NhUCq2 zk-lyw#)7>VbW95jwZP^P-&2>W#}&M6K--b;nd1QeI{fsu3rXlKu*!P_)Bg4+Rpwdv zcY!VRGmoEQYZsA4Puo=Mw3eHU7d?*ja8TDyL${V>>| zXCaoXfB3@H{G(%HCr0zX*c>BIy7sD3As%s82JH5#-s6>dU?FPcj%V2lQL!c3Y%#T4^#B zff-#SyQNV3v6}n0Hw4X31{vL22le~ezhb91w_w`~B1;PHmGFC*3Z4T5;@$bPNCWYB zfJZ=Xg;>mpU6^B|#2NBSN zf<)#o+OQ%c1;Aft2)Ncu7j^VG7Vn-+vQDl81w;Mrq25F(ro1U0+9^?sP&?0*5E*Eib`mB< z(#RXMe}QmN0jU3ga3leY2p|C96}&shL7JI3dNAK<$c8U#ve-qoLI)=t;x=;Pg*S&M zCr_3(vhTbzY=^^!m*k^ZHN!;Ess^-!eQrK0Pop(z9w2 zN>^ux2BCeTSJ&Ftd&lGQ=l9V7&JUD8CJY^sV#AL=S+K?a+y~hkb@|h)OdH{&Q?>tV{1asW&zoV_Ua}Mrc5m$YyC`Aa8 zlVjy>^Za;uz07@%)caN*7dGrE!zaOVmYdEdp?#1 z{+iUVx=z3FO`Yl!zRSKFsnEVb=5nR!Xez&)@E=cgW80775Nws!pO$kpT1_vs_HW0k zxSkQ-)6eaxXIcgsG_%iAg<8JZla@bYD^3|FC3WXoupWYEIXLZq(xdqeg#5PeP2d-@ADe-O1=Ql& zj^gy$uf7h^>r(X&a%$>(|DGo{^}5*A<>_hpN*X)Rl1CaJ%ej}zot(%$WX7qesho0DK^y*r2??mjI0O2gq=R*i2udssi%Yei9!u?jMZqCdIIjwV^BYKv^X9W z)y}plzio-j(&W5odKbabX5|c72xpF~P~_B_4>A^ql|fjI3Fk{sJqgJ{L;cM*;}!Hf zHePB~JS=5%Z#yQ-C6)3jW{o_Tsj93{S|)q1=DoLny2(B%WS(Jn7t#8q>i*%n{VI^M z7WTEsV(DWnGx6|xi2&J%-3y2A&qY{L{%_xpYb1SzcOR-a14kM@C4~_@q-REO2t{te zBxD_W5-5G(3|%Tg9hpU6Fa@tbm?^|66F;hk+a~8TPab%s)$*^V+xRM`np=WUSmp+% zN!{nWKbO^k1%fJa>&xQ=1tD9*t?j(lMNlT~_1?&aDi?wSt=2mygT-H6AusOa0mfS0 zXahB~o)Ee4qlE`O1pWZg@gHT{L_F-@gqNC;{2xX%+a3rCB4QPRK_mSKvjsiTG}Ui0 zO*V`UY1tQD>6}pGvy)5r+E?)+B$j3d$7BoY)J_J>`WG$7u-=s1E{wr2I<-q1Njtnl zE%1cE?LsJAjM&KH6|#zoDBsiRAzx zEp7r9QAIQ5RqNwdQr|3c@IRfKawQPq`MURQTswGRs^Zo3Cy50HtN5mq4uRutI`?yS zEHMK%)$%>W5JE5@IzP4Rezf#iAVliyZLQT#CXFy9T+o_BvWWX6ghxzmVhIzr5Izox z@_W6BcE!x;z|i_C+*#&2XA(rAaMQgs6cf1i6wmh?Hs~t&$9bn7MlB3gl&g)Gad$HB z+aQ|IERT`TO4t9$&D3;PV!0F6Pmx9+84DzPLz|hv7LhfszqTp~qA>TFN{T)!fAXPOH z8xIgpn*DPVq&#s^VU)|i*Uk)Wk#n>0$9SEhQnevNP{Wd`_hOK0(6(kq8K!Qn$e?;i#p*HBFJfzOGeia2KZF=p1U3VJV6ABv^Ff0J)c*4<4Q)zis z9Kc?OWy*ZV<-yS7Opq@8)T!2`Mls64I+}W}Jd8|p4(w4BQ30q0zD8gNA|Q$wFCzjQ zlfe~HrV+*@5zPIcv4OUZ?OPk;jg8_cHMADY+tAMX0*U#HZxUal7-zNs8kJ70C^6xz zMS=YWl6~-mm2g1t{ir!@UFyQyFMFL zVgD=1!z4(tK9^u@Uy}b}DkiEvpHAUGR`+2VI$mGMJ$9hz{Vn# zE>1=@))y(9Sm{14!TyHQG8({ zLfUu=s>R)QYYSy^AR0vhH8CLT{jgW(sfsgEM*s*pn7?V>uw13iQ4x<1yLQhbEJ|E&H(5)!$}qx*1>^(IjJH3&Z6?MFDFF8pa8lOKZX z@H5lV_8kJuyQ`49`3UDUt?l|vci_C)5pXUa@yx~moIA9CKx0Y+W2I|y$F90)VBTO) z8)&n?iS?5Dn0e2ngBHh?L$XjIsn(`5AC<7(oycqJ3(?)CO>UyNX4yAvn~s0SJyEz$ zFU<=G+~LA29e?IM%8S?rym<)>O)k&b#GY?y%Q;c) z;JMd(`dOKGXl$9;H&hfJg6)$6x6dKdI8Xil_8hn2?@pC91pq|$?rso=`z?sTz?RQT z)7$4vt0GUH0v)+IoibE?W>=GG8nrKY-bm6w@&YeZ=nr^>yGd1;IW|m1&eWR}=sk|2 zFSg^okgT{!-~+?+U*q=Q2;&_t65cM-aGUpk;q*mU^F2=HUD|UwU-0c;cs017=|Nyn zcIoPK>l3}@UUu<)w#be9PK)=>El*tg&3ur}c@XitU^aTkPZq&7YMycNZr>?^yafR$ zcb@OGRB4NV(*3}KBHa>{0Djld_fX(xRPUk!y~e!&HI%@w?m?-M*V`22gga0o~<_3nXXc73QnL7{`o|vzF2XlQe_tUfB zh_vu*CIq$)dX||2|3NPCc=f_U-L)W&APAmT7@SjodIH&d>~Knn zy_F4xxkm$ee8xrWy}x3=u}qSJeHao=Au7U3h`~^x2bXFYj{d+Hfteb3`f!4IZ#9YV z@Ufq1Eqt3kkah#2wa_))0Jh**(`Ij`5aZ!y2*fG5m?o~=6(E8KncsH|yGhPS_QHOn z%hqQi({0cflj$6b)uTi`HYc$frdbgp0j1z z#{1#dC$`*dG_r2MPhQKpnryZlk!(LTY=sSc!ql9s4Fs*w)buOBAOjgCUv7C104K)M zP}2pj$-u1RLavM=`d=3rPywX>wT6oLXAOk{r=W{6vCOn6t+YAnbYPxvv5k=JIpm!(nawk+$>wNXt75rK5AnB0%vR8AZ;KbhDyt?#!8 zS)H{V`JsEWdj$B0dIp7a13_W75wUEV3B2-&l5sT2P$`WJfz({4tn8!$#k^9ULh?Y0DxG3|GUXw^;-CqI2pwYnYpn$OHhRY*y}L-cm@bPT)Ru;E=;=$F7AWV<&y7)+lXV>dntYW30X&3Ya$DQH5gpYumkm9QP%47xSpbB z#J20T2qUUCYlz6I@6g_eG#o7;BM&B3$O4X+%aBG3PmIn8rih9*G{%-(5!SrEK30-6 zUFk^`ga7^%WwZt#!D!U06tjWuXAw4KYeE!l!KgTm?(j~dmgOCte%*;EA#c|kCklI^ zCl}f6aw25HY($bDKe7Xesf#z@ff!^S*+6*{_Anq34+tTMK(9MH=)G(+x=4oQ$Bh6& ztKt9@Od-MzVgD0Ub$@Z&R}>-0I)fXLJgm^15JC?gln|~%3!|vlPK1W9s)MEbKxi12 zJ@Lx9%{c*xvN?LXXPn&e_VyCVI)E8l^GxJ-@-Z1H=0;&;sjyh?Zud3xA3+`^vE~Ly zuYYJ3W?D9)i$MENUTJ>1nByf+dAC8-llz9f)dJbHc4*ZKjl%qABD8QTol3v;Rbj^N z$&h*0o?tg^igD}xl3h}!2pvMUQCLx?&Ag6vNr|2NijCKzc!&qcf~DE8h1(mO(rZ(tg`@o8It&5}OEswTWq|=qbCp3qd{Ac$MooxfX z6H)mv3rk)Uhm1@Xvtp^4QMVWWpC6jXUF9_LC`3eFu>-9lYnL7=SvbS1iGet8D+umwLjjn(|>u*S{X#vu7f64Tx_aepud*QpAjJlmpC+eth0RSGG8`$^Jd6z==c9Aa3X!1UaJMF1k~8z!u51 z!8#-k;_J_Gx`6zkF2#==mgKH|6_qxcPeujK6xei(NgO>SXCoT?#AC0~82Zb;EDjx(pclyN{-v#NHwGVguax3udbE!C@}KkvuFu?p=j+<-@=5Hm*el_h^dU0{RydZ^=9ux%ao^#;_KDZiNzI)m%4<0 zro({}=9s)o8c{; zcLe%|VwvYEVx$!bYYJeArHd6HyLCud)iqkPf%@Y_>0G^{YvK=60F|8QaIGS=m!^1I z=YDCS$cqC#`{^Co9)?Ey+FYgcDid=oKWcQsC9Tn9v7uv|7SWAICTkyyr(CPp+S|#d zgcDtdu;R}bOM4x7$IP2HI^S&%-**YHRJiyuPE7#C2Wk=*ZMq(}i21ZF#6oRiCmzKT zS$Q*0BF)m!7XNMz6D!7|Q*IB(&KK|HcH5zE|;t)O`11iHdz&v_@hc*~ zqtK_80yxZfjimi#bj-GbzM)w}S{sqreplqr2UY5#p){Kf**#7jC%TyR zefa*-CoJ!sh*DyN>w4&^XIBnwdfON!y+%QN;p`Cg3gxOQ@8$7U(`)X*Afof%8jnMX zp=^;w)oF;;yNltc^*kZjJRPr&?UQA!%&EvDF%@UMqT9FWCJ;5WJHI7N zwC{3H+>d*|TxR2R>`N&zEao2!o8~ph^y6b6A3I4DQf4Q6Jdb!QIS3K0~anAms(&qwqNFaK3?hhFB z*IjT5+oQEnbF0EdX3-?I3UhKdw-oE~P!`im%mdPHxCd}63mOOPBLq}By0uLK=Md~t z@l4NlKag-5#2C5DA$gvLxuFv1n!9O9iMvT}6B|vsmj@6B&O4qaI;Y~aOohphq=#qzhaPCn2DQx zfQMKMrwC*o48H-dnUOI|I}}(T8K~N+&+AH=eb@Y{aL{k=d;{vyv~v9A?2%wqQfOu5 z5%N}2#vsvfx#oW@HJn`l(^Vw)@QvYU2R%sqxUNm52y2i!h@N z2j2uo%-b)`n{^JF(9(ejpdPQ1EeAYIhCt>jzd$M$&ehHOhVIb`7T_tj2&w{$Rx{+%V@J|#(MB2w%0&j!~U^J8Dw~|OY zc$uXB7!XL`4lG>;k`bwZQyA#6W^1FvSOyyylf|gH8>(umR)wbQkHyqKsWy%n?zO2N zxT_g?#M+UAn~%pfwWL~>tJ|KY-jCt-^`lg(N}Cu%|7LOf#7NJ?(oEwhPZxU1^u)=^ z!A{KM%pfZ#WYkDzi_e-YmZu#ToIcBlrj}cQ$oq)UH8suzjm+F7w@nLA8c0qX9*TJT zoi6p1Hk+&|^FZK80)hFe`RZ&}b*kgDLeK!h{x+V=v7&{I0qwZ1$k&nyPM!Qx_*0pT zZQGO0G~F}^S(6eyKWH$}x&JiuD7h_$4dZMh-3p$RFGQeulMwj~*( zVwvKq6{|j`N@^BsTxR84k)r_vfG;r+6ot5>cCk8(wSjiFN0&B!%*R5}P-N7oEtWHM zEs%jVXNfFUV2Nl*qzo4=_SJMW+rshD%TWoS-UMz;J+2fUJXIHzdt8CM_V+ zD#v5X16t*!Q!5+_254mj_K`$+3<~6L*xsZR2R7uETLF0{vb51S_nh%r0Ocph1z4(z z5>g2i9YsAA>~{NZw6`=2&Q-#Lwo9);=314pw^iXU#Z?oQ$jfXO4u!0BB~}%+(x%y3 zk0nburMj%8iss@goT_`&5_+d9%=W3u5g}-E45(#B6ap0cmj)gQe{WKf~HG!cuKa+~^2f@!UFo+@xxCLABw8}qosXO%cV@plPlCK~5xxO_01RQ#*iA=AaYihSdX zgjs9QB5Omin+??i*`hcf)nXIXrf`%c=We>_n|SBoIE7 z9Xlu;R$d4h(k)g2l?hT!jhP;0osH>GEv|i9#ux3BaFAJPon;jsUeaQT(kc$Hb)Xb& zMn&0KXoVP-X@BIc8H@3KN4$!2b1 zMz^?iPitf+;w98BUjK@zG^tg8D^D*`RZp8&*I&GLGxRplt=f0kzKqs^4_X7fu0GN? zyld(HhPVC$>n@xvD3+~3s41*$ydhZcev+&~T=;%gx_)RnsEy2y!n7Vq-YyFG!PRnD z$}CaMXh4PeuoirK(P-!S?T~1urxM*z!D>IM;7A+ga0Mm1Dw9T6SaTf~=zpDHcmQ1t zKm(u$Vgdkwe=2&QaM37So_FSGi6$8x;Y!hlRmU5>Hf)_7iZq{Lu;ziMbT9$dCnzeek5@q!BD>#DS7&6AFzu#s=JIu22kX5;IA zaG)NCP}jpsZ+bqJ_qM|Ne!%@SztmazO`o&j@v`awjpmL1Q(SN005zr zEJ}*2hSZ43Vr$(FVmWRbj0L5lCW}|ifGJ3n)haUul_)4LiIcjA-p5n0;xq+-!6P4~ zfOfU01~i~ZW#&U-M(zfHT)Xx8L%UOz7eQh(mKP<+Wt4}dt94+MWNUezl@+;0juZn_ zU;!zWkaOq7C*AEi7CJ&*sgbH#QGQ6DFLE2ubwr&mACvHqu)7-pQZILm8Ym13op8KU=;v zbuwiPpfcWM0Ki_apun{=ucnq)=!(PO;9v^HVz@V03L3YenlySu1+4%nsh(#gAi*0= zMJk?cRtuFaumVu3=Jad}1&$SGt63Qn0&5t&e3liE8q-!wHqzDS^Mo{GeKDxGO~Gxa z`+3(5Jyu(EqYySkKjVltc^Z-l9?RQWh}Au=;-;U*9Fw|9h^j)~&Mg@~r=vG{0W*rB zawpl2q^S9j25sVd!xv9i#uI*@7fgHKwmLcY@#$*ZyH)nJYfs}_*B+0^&f8xqpp7p) zn)9NryP|l~7<`z$ZZ2NO*DCS9Pd*ya-e2urs^UU7l={_GJfRmpX zv0m!g3kP-e1)RL(Cb;%t6RlmH6N}_Bwzae1Pg8i$1PYRChu1fZgXebU3`JoP8*Dr* z7-%H&E5CT~P13!UUUh<7f7q;yVtn^;In=oXv@X~ROkfLqi)~PiXC@TsS~Gz0K4oG} zwGWoDbCr0~G0{39e#)eaDElUj|8W0XRCqpGI#t>Xo<1_tcn#~oeR?dha!3~&8R1d9 zhOL%TgKIlId5j`(mCOt!H>H5Io=R^jT8X3(YTA`hF%|Sy4W%B2oI*1^W4&mLQ=g0x zb1X3}T;zavTXHTiwl75@ez#m8Tp@`2F&{bR3-lm0AF1{D`DGTDiE&KK$9g^%5+RgJ zh)ey)ES4>oQlD5z?RYF^j478fmr}NPYAoUW9m={4-H4(ak#!k75bH>@=(W$R^+0FF zR_dWLb=D{;z(xY^4a@Zgky1thv0TC_(4s2Fn2b?H1VKv!h9b3Y^#15l=MNm3a%(>vKJ(97>!) zk>QP!bx+4}-PZyY@q1!x^}3l_cuTV#owNlXr`h+AhObzi(9?GM9t!Z_=-@yMs)*s9 zs{8#YE71D*7t_jMPYBz!)em8E7+mi`H8y%XF^ z^zjP5?eiGR)RYhh6$-$J7Z;<@ z6Dv;aR}SZ~d#q59m4;1Ef30)byTHw;^^okc@%f?~)T_-Q_SyPQydPQju`Uhff30)$ zFhPKdRYn=j9I}xjP8mKM-0b4()5^bYK6C1(s#u1vH9_F2_^aE64h6Wd1j6Cpzabhu>XxT+bvC=fD z^Ba;mr%^$-a`y@}l?-t{YcC2@n6SmxQO#!i;DKF&m-h-H8`d_E{%(GC+jL2!87(|v6#rP70l}c@uw;dMB31qn>9C;htuc3w*)tcHrV|4 zi+?O$vuFQ2*XpC;*7sgZbNd;T90)^Pwt5$jV!gKdQn`3muTrtk`P7`S=qPd@uHrE) z`=zboHmUjF*Lz)y1ga=1L%U;ssXgqiZu^uP z@cXEkWA13AhA98T00RK%BtZ-S7$72me;8nZ|C<38tRSnUko>H==z5N*4ooD{0{>xZnSUuivcz;yR^EtUcIuh{c9(Ft4o22DTB!Y zUs-MUYJ0y!hF(?UIFZVpfq_~1`eXIhp2=ZBP@hjiVP9pwwRWdBY#MYTG9V;h&?_3! zuKqL_W+-lAci4isut+jVPOD63T!}mkN7-m38U&#a4hIuwz@aSMNYIC%&K+FR8OTY2 z;Y6c5k-UdViG9bW+5s^mi35kDX35l4wyL{dvYRF4YK6h@#e7_*43b`Gkov5gDuW>k z*zR<{BM~rIRo3hD`uv4Jfz7zngFLKB4BhH5gQ&{R%zL_6-;*5<`-xD6<3QS<#p4xQ z%;l`-c3U+RKmG~<%Rq13Y9DxXxG^dOrKz~aNhKy+aFc=fqXYSLaOkdwCZ_9_BO;&m z*n5Kwg=>Y`xX!Do)$17ysw?UnKOwKj&GBrd_)UfP%kS-GK`fVt6U~nNL?n4TjYr=P zn0Ix2vMCU3BK?Ia%18_y6=E#kKPZ%W!P5lh26|{#BN71s#MbD*LPkYh+E>-J{0QuV zG5yefQ#I_U%yLcx2BKg0qKI7H^%EXv{i6q7=f6)$J%kXce}M#IYZiURNk^*3Q1>DS z2_v|U3kG3jCA;U_Ge5N!r|A!j6$cp6n8PQJ-I5GLIv$ckfodofm;!YPU&w*lsg@@Z zCM3^N0!l@Oq1cwm#GARm%kiK?zfmu*)e99@NusP@ys(dO1p_jBP)zxR8f zj^$_us7(nK8)%*wjzzE(WJiSFnFMC!elq~O3eO}`HZ~;sg&I$1( zJI}MQzBX=4i?N^0y#eU=t8i@SFvC&(N8ADiy(_i97y%w$HnpUFvulzQL1pvIWiEZw z3|T>}@AjZ5{DknJxeA-GphB>;ZKOGY8Bl~pUO9knRxjViv#!D?Pf}A(gmVIUcaRYT ztjW9J3eBI}Zx6G7d&1G3*HEkot)UYk2wFA~{yyJJ){cE9doJBQj*RN-6n7>U1wi5& z(ZPSbOBG&?fxwY5juJ3X@7=>WCY{-zAz(6?S-+huHaCF4^mXfdWR?gwyXpo0LaKHuyA46!Qz8VWN;BDuuIr& z7FFonchLaA{C-7|zhHc&?Wqwi4x4-E0DV2OFq!b)z%4Utj-sH8_RdHKCXKQOGRn2| zK786~+f;;MwBF;vw|r=S71L_>V2w=@an%YYchpY=uiyhrbD*eX1~bas`kt;`pO6r; zTtc1q`p5)hC|~M*ausJfKba6;n=W!3WpFX-w7rBlcMLq^5c%jw0W>0r9R>)Gb}}qe zaVPXGvNkV}F#vm9v@9R^fS2@3szh3nbI}nlnGpM)a!ewb7{LfR2@L zjK}nV#$0uOt);@$PL^J!lR{-^Yi7!ExZETcyw4ykKdwN&Q&D$;NQ{xjwf|AV?T0O)~!Wk$YI_Wu8O%D#5( zOF6mn?9~QhLo#H}o;G<-+jZ{Tw&8OV&fdIq`OeV(;`U`1tbb+Ufjm?dDMq67cfpD`gM)`}!W#7frA(uRjPHA+T@AUpPDr znN%uzETF%agkHyDYplL-6ccM52LI1g;dm0hpx$dd3i$-)Hz7pJv-{#HM=~dd%5mjF zsZVJ-FW+d7q>{O0j`1@c*s74lQiVdPkXKN&5G>_JrQ&$>%9UFB3W}?dCc4@}20t~} z$Q4S(bg|6}0_o|hZ6;g!nd~98>&vB1T)NlzTo!vwGpX&0 zXah=r%gTGm?hRUF=yShXTS{eBnnThnH$BzO!^zKjc#}GdbOo{ao$%Vem>i6!xjB3& zc;+xBwY2`paePmBm>=En6+R2o_W4_Wzaz0D>#|L@ZuzXKj$oFW#{>t9P81lj_58Wz zd5sKQ3mG=So(lydN!3?`O4P`VxXS}(MH01;?prbJgBtTxjFUx=-o+#bfE*lIKH;@O_BrC@y*X)Ean?C`|GgF*!Y6_JUo{l+ zpG=qC*Io?OKPOh;|6?yEB;cPDt0lc#=PrZk2O7+pPh`Nx3BX1tdfWC@9FtC}>rssI_(ez#s^a za3zGusMy@3!hE7{<>jTrRaK}BR^Tlet+R8I9WmX#B*bg$`0$jJgTwWHLBd(t-90Nr z@a5aTF0XF}hepv5km|-K!9nMR+TD*U{=#C`8gO`2MDn#gTWJRPBBRI}lQF7Gb^%zXL z)A|1WxGGRnMDK68MFbV2ax>(VQ9)BrKeZexi&|%O-91&bAAk;f}ZHl_w%N(^hW`biECa>-A(KU#TL}W7y_zEMLs# z<-0{wK!y?hrY1&kgyAR6_IpVPOv?0Eu(*aOy04DQZ&i(N(WakB!9NIAfv$y&P(Y{q zF^W~4MGSQUk+u93y@>7pO1~dZu?FtU2K)oT;(q>?P2@B8wHn zaHBiwC=R0gmcmJ#3+GD?Z23Yh`{Ja2jQldW?lw>N_9sf;GN$mQz>Qs4cFAy=9csgg z$hUu9aSJ|@Ah0vKmpC@=0ZB81xfOL=3!q#N)Awa$YRmOQN)k<@u8=;pjPHI z#Wl?5)xx$^!?8QU2qQ>*&G#ohpC1o5O&vQ-wVxg-$aYhNnM-Lm`*RW_4|`6MUzXuk z<{5!Vy_F%>VrlK_4ccgJF+1-emCPWQuAZ`n`L>bzE3);|xN+usS=(fxLX*sFxc!e^ zmaHS5MRo}U&vl&_CT(EuFXGPHw4jE~nv@t~*zXyOO42vp@wmmPUl_3p^b&v-tbjs= zScXBFq$OQa5J7H5ogH!9mQp2t*F=7p>3Wo@`Se+(lk-js=dY;G>yWMKB8zgT4*kT| z2BTRmbowC_9<7_^A=oap8GDa6>t6p|kEk5s9q;S;@tgt%pft-B!#olcW$lu<5TW?H7iFyDkzByd7^JGmTCoZ zdR*Z@BqiKM==HtPrlwp5vk3TGs1PQ2yrm zyMN)P*TRO=w}jj9?;EEUaon*)hxv(HheiWTlL)8A5H#6RI-$uR%)3r$VE(6<&+rgN zWvb<9QB_uIKXmA4?R1pBJ-)l}_$$i0jh?F-p0@JA{sQ&0)lPniehypFMDTIrSz9SO zWE1h94q}zce@On_h}m(A{A*JG?uCQxFl7?gWl#(Hu4T~v_boENcM{dUIQwP&ZTOr& z@ylatTSUx|U*Z4|0Hx6fAYgRLF*5L#I&N1g{-mQZsLCtV2y@M8XI!2sv6_uDpHu#He zgj_GwJ0lAZ-GNh-fP^`j{4x{st%+w2+D@}v|*^;2JRpQp|}-{|npA+uRnrMF$ICJfx|Zx8GL#5|IWUbqEip{Uz13#5 z0L0OB@S0cqNEnHT`k%-`$o~^QhW*c=iuz|zL44+s{J+A-v`_^%-Z$S`^Xw(0vA-5O zQOmUnBj!{VC1qvO=Bw1A$~+P9nURGT+TuYr=+TBy_Zrhq({gt{A zICY-VnA32Vms42V_)rY;{tgKV0ENY;s>>A&szyY`Kke)93!=RpUWBri=?y_5WSs)m z6ZP`%$7WfJ%i;1k2f%(JYe*6S{6QEErGS!71K=5`cKb`z#Ku!Va9C&znmmhnJWvhP z7cM|)FeHN2{UKKpb#IDT)Bsnx3H&5bDPIg5&d>%tlQ*D-j)+KVjO*uKDmu#otU4); zW{1f+E2U}$m!;YJX6tw5V%k3l(^e|f`dJFaE6KbMW+Q!^>7WD?YGEq^dg_7Iyn=M= zizpKrx<0N0rZ7xc-+WM7?P{!!ptC14Uyf#b*xk5~xHQ`C2FCOUT{qA@ZqpMaL-dVFs{ss)n5PvuV;sK3I< z;`oB124LVn{0%w;hT&*R(0~A>KgI&+ zSP;$zVYnmid(l_}5``g*6sL*+w}eN`?-c2aq(ABZuy4o(6R2-F%UHE;yKLl}E?Sui z8XSe1X|8Ko`^vg%Cd$ne;X6*N7OP2vh%o7;v!|DCWS-ZX<#;)Z5v_QcO@yi1^1TA# zb$vEc5|zCEvf~5SHN&V}4PhaRos^W)H&WnDG*^=Q`LNc~MnkyF3{3}IkHrha7-K{k+FDZVo0 zUapU_dM^JoOpnPhD=+xFVODt(lKHE`X~3$c%Cdhb5%99T%7FB_9kk;I1c#mzK*ZHB z2!yjz*$)SxQSAqDOtcY0Ai7JC27^J--S#5qj44KfybqG_gLVV!`XLrAjQhXI<16sD z#u{7K_``CQ>Ggz4>DCYN)lnJ5=tYrfB{E-J7HLae9Jwr0JX58wS7ffWhyb+++w4O- zE3$W2zVM8!^AqMefiumbx}CpYvFl`EX`Sj=y|`KJ6G|$aif5m^lS)!S2&$URjkTLw ztXwC}EM+QRS(mE1x!Ee*BP%FDnnO+Sq_mt+ELQ3KWO1fqXKd)?*rh`nHJJ*3O{-HVaCuy`S&dj>pPzJH|kamQ&Ckh-r977rp(@YOmcL1CIXB z^}qJIt(zh@myMlF?+pCb;PHas!L@q?WA3Ysgng;%dB>LFAkfNd(PgEhVTMSeo7#%c zjK>9_A@X8}K@+2P=*(B;ao@-{!B+f}8+0A&BnVB2(P+h6P!buXD6;lHRj}znKB5)> zT)?5HIkj%gE4Z-h;36s(4)&$Yhc!3tjx&TqAY6k+5WBVBJtN{>`}MDD3UDjnYDy5Iq_#Ap>QtwYN;HtSzQvIbLna_^@PX9 zY(!d+8O`sA0h*S?5QiXBrHhfQfP(Tocz9#P(KG2#(8fTob81db%OTf^y=;2nOcsph zmGV@^+$1X+K<@dhH7IMQNaYcHCr36Q)>o(aiCf2hhAp)R58KOIr5Is84R zMoNAwSQD&JIsNahE@bN>C=pP=PW7AC2Tx(mYl|uX$Uk<}t2HND;+TsvV`KwyEXO zmFo78Xp0os8qckkZuH3st)J4V6H8d@i9;IR-IRqyZ9G5{{hrfXQcB7+$l4=1SEISw zzQzUCMtkEfGH#P(E{^*PJe?mQWQkvBNgnDjt_umq8zoaQp zlo7J;lJLwiGsHhTOvG|UD_kds5o*^cIqAd_lMd`NE-%&Qj~ng(I&I$k^3Xxtet7ef z^|c8ych+<#Gz}1s5{Zb9z@}nONn%Y+XUSkr&dSWi%E?E=&`l}KFUR@X?x|JQ4~Y&7 zXtrvMZD|kI?eJ@E{ZTjM!9NfjZ9g&EHQ6+yf!d=sKf5YFyz#wZ>dU&B8r$pps>50H z?mynVqD?EJuD&Y* z8P1jgf0+^BC0=(@VF-*z*$o6-ZNOU`cCIDfOl=wnKme!oI_>}fVIih&lUoJ*2cFph*?%l?Ce;TPqlqh6`> zuKJ2mf-68DJQ`JEqk-S0IR2<48oU0GPaK7M@Mfy>rpTZ~#O5rnuKA%dc56nW_Giv| z5hNAjBoX3y&}yo*_p16q&*M}pL2R&8n6cuh*#2=gPZHSieq8RmzT)@5!$es#2^A%o zh(uhz%0CY$lmR)6W#l2Qj(yaq!6!O)@j|PacB6mUb!;aH(y|;UVZ?Px^E@f>Twpa; z%Zi6B(=1E-ear#&dwkDj&b|gXM+~a_=c!)${iDT6dMy?z1}~aN>%Qzc>w>Y|HD?r` zajuZ$`tgn%oT&+BVwEPE9_Q^8xeAHuIn{*hMHlUJ77I?jP>(H(cqw3oxTux51WkGL;84O|PS zRXtE2oYlPyQdD)L;M!xht+_s#b%4_JR4Pi2NvE=zvh^-1fzk=BO{Zin%1!5HMcB$x zAZDp0R94`trCWXrx0OrN#7KFCT%(qyYwbwnb`=_@P56*Mkd}PtNa6j$8@m@zvkPxF z^KulQk=C&X`WntW=yS%yHk&>wgmNmD?<~0uzK}lQK&6KM4})E$1yk-dD(&%f{pv5< zn?{}|-Fx7BPcU-=i1An0gaI#$a)5w2=OXzqma4uQO?F;+*xs11_4pojt=^Bw!c{Xm z6D9>)ZECPRMScQKp7SEKH`+ZK*Wswta?@$Y$C@WI-f23JS?jS(Qrzl%9z1(?bIu5l z{ugbF?u739G(yZKqS?vP%Oy(Tq+Hu=6d;reo^Cz+Oh766gD=`OzA+o%(HsIgG8`zz z84l@1#MA6U6{yyceX!;z(2{N&L`9}Q%Lssxjjr`+7tXgpZx+JYV1)YL4l>w34zf7_ z4n+K42N~@Dc94^!mA*V=b;T?qRR#uadkuOjnUw!&kxEgL&CF`f(#g&1U?8xU`>s&; zpBAa`w5FD+oX*)Udxz4o<(1X7m4?ZwskXUYRr;m%qvMm)h|TTG?DpL`Rs6%V-%rm2 z3R_pdZf@t8OTY)kza(<8h8Jq*C!`!sPbgBUE-29`p^zQik3m%9g3-96M_rv$c~VwR zDuD_eE}26KUmV#1;qiu&X($x)xZSl|PAC;(K)w*69JJ(w@>S@tfZeVF> zm8(=cJzZMx=uHX^RL873ynsj!Q;&pp?dN}yr%h&t)w`#3b-&g%A3B&8UxfBBG z-clvYZv_F`(Dg5j#$NEVZa6XN7v?)Z6zYy0H4vs{G#D*hm1&T8qj_x~1geedAe!at zxe~?xtBf?3>H72lB=F_MC{76R%$P5YKx6+iQF+W?0}jl9DnCqF>2%LcRjNGSSNmg? zJZ%djDMzn|UUT1`Z@%;>WzAGGI}>@}j66l()%_@W=#ULF!mQ;P6w}9{i6ZmcjHE?= zSdXT8sLjN>S(*oK8CtFn`>K$$8H5TMD3=|LaVbaOd3i#iFRn>K0c`qtoCwJ-u!vN5 zPpcSa1`wu0h2Tk*6}ypcQM*eYSzH$#x{g|fr6@&fFFR3OP!oJFMF)(4lr}G1)UvE% zEblJ9Y^dWqxrCoHvOcX{zjwRJKBIXhVA{Thsqxy+DBdrr+~1^$`E#yaQ4aRPW7V7A z;u#GJM^<$^0MF9Z-*q*-D}sTpZgclJj5#QH9o&a`aZz*SiElnepxsGNN34wn7@A(5 zTyDfKjXoWnpWuBMJ8pQpW@JCHhnVbZr^A@yEc9X+Th73Dnz=9%j|O$b_Y49N_*|*# zW`X9lB6+{=v}9xRoUVw167;b^;8sD<#z&YDS$4ojykJ#US+YOHr>*oc(g*3k;vj~i zi$7hm(znILME+}=g~dh2EsmvX?O>^&Mg7q_4MW!lImgn3FpH_9QS$<+i1Yz5gjhu+ zq+cB2tFya2b2l_QiEA$m>J?3trd~5R?Y`iItVfL(sJs_Ue>TY*s!f|jtH>e?|tM|%GR<@WXx#d3{u8#rga`{;I1BD&>$Tpo822H@58k3h=P zG5Yy>?FLB-Jpbt3Bl3CdoWcJH%ReAT!oQ}?vuh9lgQTzz2;2B|B19?J*3S^I(!ZNe z)YntvTog1KkqePeIUo#i8$ctJzhh(1gG`YZ66To4a7rrNOAQEk9NgPxU27#AkPRV1 zH!2|64a2~h|EcFb0DHqODl`oOu<|I_=}{2IRJ)UEcnHMA7#$|^ofqapKENJd6CKTV z5aM_lz!g;=S?gxvcak{3*ERoIS5)Mc8k)nm$Q$8{oPlHQBgFn3BjBk|R303UWFBGb7fS!EZq~|LvDx~J*0?&Y^{)t=Gh_P%^0e))qXvtGGD%R^utwdz zB~fIwP+6}%tDc?nnl5d;h)bC^FSyT@^h>2hV{s&oK%J|Tqq$5BeXp*@`%|F|%F&tb zb=!U}597}mYWL@dik{Xd${*YwL8k|;897bd#}{kOUOo)He&nwrcdN1%F?(3qzPd1{ zJ72)N=U6+hW~gm1Dkh73GQ-D=7qkVk>0Z23c{3?RGe2-_|Elt zxZBZOR1xSJ^jEhXS6tL!&7EJ=_CeBK)(v4=U)GONR9-erv0q#^&WX`pH7%)GUp23p zRbI7hxn5kg?uOD{w;iThU$>u@RDND}T((?XcixWBez#>@vcBnl+2Fb9!92dW>3zPZ zWpo8#+1&O)FIwLYKu})Z8o`Tu-wvX;lHUy@s8`*|W4poMju5pT-i>{P+Axcdl}6u; z(}5M;PrkO$v2Zc2!VgYyY8y9Bb3MaP&G5X@vC;~F@jcELpzE?O$jBEo&C80*jx9<{ z+cx8>(E9vdF*DI+TQ%_8YhE!->mFXy&#ZPuwMgoI+7xZdX5X@(L}1%?+4SM~Wp&Zb zv19s1z`AP&_k(lK5YLx$Uz>J@M!SajSlzmZ>qN;bhxs%=>Id_g zVx}+mxlH9X(?xY>EYqdLh zSKZx7uF94kX<0Vu1lQq8d)4#vR#$UY>V zZ)dyg%;tR1?4$v$Km#ip3&FE=du%HBU|7Y$0Zf{kkb&@g9W#jx+zkbH5|vQM?*4v6 zyF*@LR-^r_ut9@UMoiwVXe4R#E(*|t2)@8&0GK(iqQ_kn0nsuKwtu)7FaZVMz#kK? zpnQ!B(H_y=AOA#RlzZcC9>m&<^yU7`d)c41^9A2p108QGML~6n;vplEERonns7;9f zYWSm+9GB9%PXtaCQE-fp%X!`><-r$IiAzo>W!xv1$rRJ5k571b0KjXb$?43gC$$!s zA@A%3;BCgGwFa0W-7gU#cm<}6a36qU?*>%WFK~1f5dqrxh3>EDo zS;@@dxDnzgOzuP1WC>ZtOUHC=>jXBAO4-#0lx|z?lI{&cIR_CXRQ))p4mn9AN73K- z!+_JA0i$dyRq%Xlmb0q6i+ry-C4#F-als5``RPr8VjP|!?gIU>MBT=i!G8k&WrWF= zGYjJUkErl3GxGbFH~#ND_&=GEU-KaB|1l33ADn3trjHuAr`i36ZeQy5nAUGzcz-P- zzd$$!1O{mnlDmdcL_~$hQ2f;QB>SiEHH8oaJQExuI9I~iEscycsiZiG5FV@=%+EKk zq0!KVqNoiB$ka_tCHVzvS8%=Vri4HRF{G>@D2Ke0hz?Al6!FJ&oq=eS&7w( zOpXkR?r0v2Dii2L`NABTKP;fk>VDC|WE?Ulaup`tHB}pqR5GSnWE6Y`%38|Vyu|UE zXJ}<}HZ|`q+LP+z3rat;L$PImU_tFP`huB{l^~M2^A!mws0TH$i zVxhRJ4QEqAJXmZSt-k|$3q?f;w({A-kjUo@TrE@U{iCoB442&F0`3mRl4$kU`UI0h zkw7@{pu4sOgQ2LXc6_@)fJaGnq!v`TQ4tVVvw{M;#qA)f0?zFlWC*&cw7^^6RT zS^&lrRz=Oy6qcsUe*gkgqi!Hg*Rx#+Xq0GK0)eHI*Pmf+v|ExnqodHDq`2ZyoT8;R zM-DWK)jUquEg3(~#5TD#R&rvuJW0`H-zYO6aiWi#r#SQixE9uT2z!_NmF6f@~0#}Y~it@AX90*yigN1e*nZu zF47v<`5Gx0p_>hzY6;wK8MnT>4d}3ihauQ+#SrBGGZHE#vC& zg4=p-Z4B0~mV~q3v~NKcx;)t5VZB}&eh})ZHzjBd{A?;AVUcnyOD?c^nC@<{`Az4z zknVh~!R>Q#?+=OVJyYek_^^@-if}eJlQVvQHR?3MS|(rm{#LFGrSDRyZjR6}rn~sc zcB2@l+jGk;{?(4jRQvZ7WRhLFX^E~dbN8h{`O5+CUwG-$8J_uSViGC<;{d z0>?GAtebYA{?Z5*Ma6ivsk)cuLfI(WQ9z8xG;L<)Ime)&Oq`DZoK{goX=R}gyxZkl z4w{#G(_O-}w>W+dT^%GXMN6%kA8h8+u{s-#+9wn8OH#`3my?!H!3&jz*F5zSk^!f7 zMg!1=5m1DszP^AtGUTN!aKo}_o&`pUeMkHe*0MpeyLgclr(FPyrhsrFV?d+`F|mpU z2TC1dV3}8btBHjq*Ak;JXEgpOw*>k(W)%5OxZIl|NM6-q6rUkw5^N`^o+G|`4-3zn zbt~#{T1T3wRpWqfv`542AGaEf3rUQV5;FI96Tm9R6pq0)R!(jf-{WMX{4(lr-u-gV zw~Ji%R?8WI(1nD!V3T7R9BGlBv7hkZ_yeHaoS|H0c}SV$d|-;gWnT=VM4l~5UE(q1 z)CK!eRzsT4P)S^-rIg?#G7(b7er&m=)K6eE0s~9vysq7ntNtb7eTG_tc|>iNsE9Fl zl10}y+{p+bG9%Wr=LJp;WjC*_{i3GT>4FZ$=>5ktMrl#;Jz}Sr^Tjj1%|AV*f3Kru zY%8!Y$Jx9oqT)Ud3f@6qWf5KfJ4`ZXydX5nDt_u0E}Wq-koB#F)itZ7sWVURN#XH$_jGpM6M;`B&fal@#beEQhQ{mt`1CE`d$=!!=E1 zltmv*Og3m(Vq(q-84vs-Glv>w#mKJf&`) zAczdEd@~RSiY6^g5e^-PXc*o+kPH~-D@Xx2jTb?Ra0-8Z)aeLt@!7d}Owb~{q!W2) ze>@O?KMBXQrHjh{@65<+$IQ2_Q@PfT)#+>3&bRH`*w(HS*&ENzw_h(atv%1vH@7%J}9}{K)m-|X!*7QteM+Tp7%ZExVHZkzP|5c&$f+lAUa6Qrq6)cOVD2O2ILu# zf_^gnVfz)W8Hvq3O;YQWs~YP#T?9E>{~^pF!0^Bir**W}HbGILob3T`PP!$@?Fd98 zJoOI7Tq5q^tpN+Xq{`;m^@tTG6#67oQSRd-d;!>(>BT*Doq*k)jR|u{wvV4TH_$=p znyurlhh|*oqnlCBDF4NJ)36!*(_C)MX03FOnB3sedMS9sP$;&Mq7%qx60+KC^#};KDGBNB66wx? zp@wFL?(Xhp2M>*>;1g@_51_pI?uW9-(9ct$}UuMBjuCdRLIQ* zJn4yvM)SU>X#b{arZn0gS9fici1QV`_qcf2t z{=zF?wFvWt07U|3JV>7=plAoszcFDATETv4{w|%4GYrq7fz)sO#r>+ZZ3RgE1sEMY z0meU^ezWl*TjbYdqA{jG`{B*^RB=1PE_#b8gaU`C%6NMQ6q~KPR1#Ixgj~?4ULYza3 z;~DMc{Os%-2eFbX;|1*P!w%`Ks{KKS)L!IX{vZ;TBjPP(BXR@hg)W>)kHoE%M8Ym+ zMJoq-B#xk{_rpB#$vt1OeLRklLo)*jIywU^+AF|D=2SVLS;Srt;)Op9pq~eZuZr43 zQtT=Gr~{q}Y=?0=1aZekXo})ugr!)q`MonF5=ai`oDARl5k796N<$UI6_6sj7Otl( z6L66913NjH9aw)OgP$DG5Of;&8)5mH?S9e-xTH$S@$1CmniL?Q}>XneP*J$`(k?PD`!rOZSlS^;s&9 zSonaU51TTf!9*7ClrNu{-`ri$0yaHEjh{FrO!8P;GDj2@o2icjbjAr#cE~it$+CEm z$fC&fozj+A3rRQ542;SN5w^%*&wQ{?ugZ+>jx?zdkE%L}Do4xMqI3|gQif*dST^TW zQUrL(lGlXAU}(jf!_ylK!;sc|k7PaxVm=V6h{*1&FGIA?8B+603@m)+d^!R_VNz(s1#>M@|$Jak>lI4Hrz3S+&nLhcbkGTEACcmQw|K+{RSQ7t1 z{y7SyMbcE>8w$)NYxSA$E1)EE4gUF0xk;b7jPs-oYtFiUFZpsU{`|0Z>8g4AI>awL z6c(-J8{`1TiAf9-NlAUlpO%uum64g6nNean0qawpu@4VefJ{wKeO&rvwQ9NHRbE)Io3>wNbF_PU z06MqccCNe-{9gFOdDHz-dBA4*sc>i*IsSzL<^*!vLB?9k^P%tv7Dcs6DB*fUOf6lu zVG~IV@(9ctnW4!v_E)G(H5EJC=`>!;(s-)WJ@Pa`>)l!7MON}cK1cND9PL%vrQ#{* z`ll~v=1Mq=2wcHqCF5`+zITzGWM{;+v0dh_n3=K5joA2#=fb~sR$CS3O#*=skX`}g zN2v=(MwQM$@T-;Q#FQTYreh*#2_7h%Z2ALeX5QygD({qMSsMKHT(L&Fo2*Y4%)Iwz zXemp|zFg=Q&Y!Q_2;uF^RdPZioW2{W3I3d)uwg!XrDhBv_{I@cBvcr2r*x2Q3{=uc zf1DkjP2!8@6%52({r!8$fVsEyjtptvB;g)eat@2?G~+2s=tJDtTvU0`WrU*o7T?%? z^-v5;8wvxlW*4Juzu+4chcMqym}}+8`mILdvKys_)6%WQhf_wfm&VetHy8Vjii1)q zpR;x`h9|<5QcU*9%9PPC_myP;0KDzJB$>a}Hi=jutMZf>5dZsRVxjeY3_^g444NJ} zjLyl&wZta*HTry6Iwq}nOuUgDi=`82r}{;bn#2ifmVC5hIUf3C$$q9x4+mPhJGwDT z#y{68kJ6uD`w%logI_jHP6N(NMO^D&;G?9{1Bj+}<7Wa=|HqN8$PKX_?akc9QB7e6 zaH@V$=x`DWY4Iyi@v8jM#BSF43r>YP^H{F;PbV^m8ue99G1T3HtVm zZ}7YH18j`tCFenr!6}Rn?NMZkZv-8~V)osE8Q+mR;`*4eyvQLlWTQr(MDWIpVfl8PLrRf z7|U~LhBY1XWX^sYPIYg^T1)%0hm>vY13PzO0`bLn-_e84J4_jlyY~GcTMa-5how;kysIV7#&c$Io~`K4)2JnzHLLc&qHpefmZYDePj z9Fx#Tg)5h`{}jb^IfbA zPih+8`mGl%K)#Z{4u_5uHmr=-#m7}ZideXnO5(A~h)ez6=Qtq=kkC86O}uYEpelL|Zdo^TOZbWth_K)nKkF-W&=h?M&< zIbETuBNiCkE7kk~K+%`PHuwGZ*e?@jgKqa;(0}Pt6zgg7zsTFeJdG8S++Bb@E3@g< z7;oF{ewG;SVoXhAq@02UNRj_iQA%$7N=dpE_STeTjP8&hINX$FooW@qjs*hC9>nm_ zG98Omf5azmaW<%z`6>}LYGXc#$-HmZ>Z3ox-Ryz*QeR5uy3E5fK5t+hSSqBe@KMi< z*MpMQjA(iNqicw_e)n64BTfd59$SGDI%bEBOv+&6JdZyyjI(7@frEO_4_2hT=0je_ATS6l|aT9@HE-3A5)( z>_(E;l@+O9q9>bG>gCn+Z3?*KS!2>GWp1koWuCw1eSUDh1(MXAy9pA`$bRlQ4KUJ) zuZk;W`7vx_@TK5ZJ-GEWG2|gqfgr6WzP(VOtX9odCnFWwK@lkvrtY7sRl4z;K}-@7 z79G-mg?y)g`YZcaiQ4&?@O0M;GZ%JhQ!Rv_ZYv1C=7>^4BT(_CBd_4y z0e>zUL#jbLIm3m#dfX>33%!owgGPb?dv#f&p^2cb9vj`=^g{g z5*4r(Em9-~1~%GjG;LXRUx+WGOSGjT-dr-Kf9chd%Sy0ubu_^wkXJ%_egs0^?7}MF)RLfg#uB8}!BqqqpeE}psLIO`zIn7OVl zVc=@QQQhaq)|>vhC`UDY)(P=e!9lkSTeD!OmCS~Ib;N9)_JaxL0PpPwjpygk3MZpG z1G3y71=qC4lCPhJ2?joWbq{x~n&#`(8C)a#>F2Dn%5!)3&dCBrk_X?mdshk>_U5gJ zenS{Zoly~q8|>O)L$w-t(hIW=s1f=FU*XRMt0nyv0BIyXKzbb)z?$U!6S*+)d$ernb^Fp9hda zt2aBT+mmtae79N^TiSQotBXQ<7n_dH4|2Z2wF@T$_A655PxjegcVo~dd!=JA9Pqsk zn5A#NP7QS2pN>-i;8_6<);}D`#C)EP;PuYRGTWD$7@b6^hj@kja6ZsTf-aR@`s94u z{K{aTIABjcORthuoh&W(gShd<^Ff!2v&?&cZM5&$bxB*)NDquxg|{>#^?)u6+qJQN z2z$yarMp_g{A&U?i?=2;uJr34ME_Xet^WD3eyGzm!MoM!Jfv(_pX1+>NH|M)g+{w7 z`juW2O17Q9FNJ(q^azPv+><)7ni5a)C%7r|?!W2M9LgtZ_6Mqq`4d z__r7)7EZesh%Up%>aXmX9B!%<4sY>Y7xAwzb1l}3*fR)#A<=J%BcgVN>PH~wopuF~ zaF7;DQI=&XME@K+LTW7{pbCOcp%iuJF{$h3DyVzHs!L?=5iRN&;K8hD^wOeJ=3L+Y zK?L&lKDfyqP`m1dpX}ux>*cBJ&8_6)r|cuIuhJb6$m452o*+nNH-M=B>R!DQBO2IJlsM2HGa(@SFpkbVd;U{4S-L8NfcFYWN z>nz2%90Ti;nCQ%Q0jqNA=w$L^Iq8{Y_{oUtBUXH!nE6em!KedbJQiUK)Pu^QA55}4 zK>}s3;#=30KWBR``A2`+_q@>pqd>`NGQeL}I9N5_FS4Mov17bJP>*V_XysU9l^8#E zINl&%$T2_zlHzXU8zq(^BN{6k6N{*pYBt1Vn()#33<#Y}q2++5Rj2Sf!nNK7qVe6HHfN){|q)8ckSNCbtm+y$(=xstlK{#ju7(eH8Z?AHTk_lQ>z}Su$&ehPK0N+NXBrlkE?+J_4 zYTid9^p0+Vi2y>={2PL7mg=5CrwyQfX~ZkEwtxPlR%KzPC~6@6a1 zex-cjbw@#2fXyODF4|Jssxhm3Nx^jv%?3_hdUF1aRx)Ny(cyMc-W34~3M@vg=jyH$ ze)iD0no_~bC8DY9fCQH!;)Ld z*;C35rubXQI!(eBIQ7bT+g-5sh2CzS)lNa(RB7T`v`;m(N)$NQohImrg zhi9?_id0xqA=YU}kW6vRHM_`2wj`NJt1-AgE_Pg?PQst;Xp-y-G79GJ5$R@Y5N)pt zaabi5%hcxJPB#(LbL~+zsLjG`#_X+PHW;_`rH*W* z1iP|Tx?)0}{Nu{lV6TuqGT>BF@l^gJFCva|u^7OcTxc=E3-xK;iy2-Xe-A1|#Oku8 zr7EYvkIR4Fm3k60^Bs9?y@}FwsEU28JQzo8E+-Cc2s&1|%VlFbI!RO5yD!GN=Q>sH z+n4n{CKQP!qTl(}?6c;JOF?oFS~wONcdv!tvitx7epag89#&1I*FX$dcG;?m+T*s3 zBo$8J-%P3^?F9CyrOxlu?V{cq#0q$Jhbs|4W3GXmgNW^VFnj)Q!vt1oSIA<2LgZ>Uq`+pWP5?uJkSHcJs6Pc?D*6 z%1jmhms}O?ykqSFHHNbhIlcTHS*$H2iy0tOej#wEX`pmEJhJCTFtioJ&e_)COFD+V zf9T9J4;rJXNF&X~PVJ|V?x_~8npQ7RO1D#*)Dvc#L|bNWYKF6qsA#_BDaou#op4?o zKg74CCp6&9-tC0tCH2d!svMOk$@v3hP&c*5Qx(Ni*WGH5(|G4iY6q8cFNTuecFuG7 zd%-M|Cd#MK9`@HQXT9EozSny~T5Zj>QSIhB(sXwpd<6&|jmd$=c|0b#BoamJ%f)1m zfo4MmT}NcB|K8)uyytl(z7)8!Hm-MH1}?Y?)|QNmm>(VAla>%7idx`<4~{s0sFIP! z{i*!L%@2cF3?)C$N`#3ADQo*rI0pSZicfln7!e?fjRB}gDZR@ODRk(&W5M6N;ujx` z*w3Y~;|EE(`|;HB2!cw{;RK@NBO)FnZR;hBzlRfz3v1ejG3rfT1_2pnhqdwxl+6ZN zrEJw*fSUEAbhCI#Cx!ZIMNH@5JI4_jm*gadQCz8^a>MaP6~dnAVgVNr1Lt@*WDNB* z{)Tpk@uD7Q4(3i@1*)8B zxfaV=FIpo+?5l(ySf7FF)kaXK$dO8+o3H`mX{EfV&B3NNsxGm~+&x5c`Xg644}QAH z55%A8WPO{uH{I`2HZxjzG2^%_51jM_2qp(VWCl}jwCXITz0};2f@a%OLdB^?giJlY z<%apr^5jc4T=X_slZ3CQ`Cp9mSt8Wv-idxZ2=&E=pylq`jN@oaT4bBwI<91Csw;K; zX*s^K>Ja5F*y>IHIV5IE_h}fZcIe}|8YO+N{kEIIS?ux1w>1wj;1)Ua^TwIMO;6c}xJuki}w zM)%+V%hlJH$(O_Al5su-$`rj-U?Q%W z^HHl$ssbY#GG{;6Rq*=w2bGZOT#G}3uR2?=yiB>4zedw^eD-p_XvQ(u{eAnvn#RL`A z0U(#W;#|e&a`NapKWr@`ZjTHicRNf0A2bYGmN>XV{o1}X&K?f@MQe6| zykPhRp6tU=Jonm(+d+SR-)Fbr@valR`ncx9h3k~0`d#q(9C%fBDU3go(MK02OF3*U zA>OYeu5{X`N;1#-6e4+MPX13E;X=8!1W4@}Puh*cBpT`weY%C@BWoWHWeM%dA4y5f zA#&~&9#Q^FTl+nCa##``VLQFnUomrfm`py!n1h)fqCv{?yL=YP@#W7Oej&vboi#?~ ztOr|>uO1F+0XXeD^h)9**ofBTYAG|oGn`QhUSdFn&ptLI#}rXxVmNa<^Yejy>8e+8 zX@PoKn}Wi^I_Ry@Ic27tN1boWSyF>JHjTMo`47BUNi{{Pe3r<+f%bhn8Px9D8&S4t}}@YYYM#|(PWAMYRXn&)Ub<-REA z{BoqmAgAm6Hz=*!AH=)7aE;2vRySd}n_5_~7St_y!TyHCxUSV$&ETg5gK%DeY}l)G z)xYPTNjH5};}zA*y3eaM&{mtCT{dT6@t+XpDDs;gTqWcCo+x7x>GD>DWl@bfh&&({ zYpHkq_oQs9J*vt zH35dSG9A`_V|7;8s%yj4znWv!{|7w8rn)|D@~bubW4)c|%=#SkS6eau|A2=KXEs)P zL}&#I*!cw54F}Ntf^nYq7njDG(P4T9pUj+eF`fpD9zGZAem!%3=%{S9P7cOL(HQbm zMTLfWFLf6QNYkD9xK^C<;Ah&AnYW}j0AAjiQNe2JROIim6`bGWm* z0mgVOkwx|}+;43faU%x(uw5X1`>MlLaMS)ijogKasCz)*_Xl<{xmqun&kfh+A3GeY z!A49=2e1iHFrA0u!+oMe{AC+EV1VbLp(juBq!Id?pUz2ZQsIj}m4QCf%M_|9lb+Nx z0mt_e)o(i~X?Y8;PY-T7yTP#C^h&ADBk8m5PXSw$#V9EUS&R+M)N3?U5n+OZJT_v! zGG-cGaS&}BzEU*7jTT3>drtEAn|G-J*TMHt2A;}z4(%3f7F(o=s<9`3Ac z?>kc1vszd<6hR-wUuk&P5r6oL)3^=1;w33t-%Mi@{m17_Cu@DS5OW{d^dSAILp$~d^l-12k;H6DF zt&nunE}589(SyHh>po-EdDG*b!Jl?5gpmk)Cps<+R&}F+ihVb4l!KQ057%(;M6ox? zxwhrT$6OWK5@4~q>ubi+FDt)<#|QD*as9m!G;T+5M@Y?%t62UIkt!NhT0tz;_p0>I z-G9BOwJfS4oDq}eTxhbZ`h=Ew(!y(;Q1dXU`M7rB_+@3oDqpDw@_B}8mIqd+7%**p z@%=p5Z7Eh+JuZy&DpqtjP4vNg@H6ctbB0vtvOfR3{D-;lFIInqAV>4=;X~BYcQ`Jh ze#tY80W2Z{2v^HElE=)zc0XUPh-GNj;zLo(>ysV$w^*Z&eix@xjuiyTOfWP@4<)3I-QT~9_-*ek?hi9d_7-4wIX`-> zM-iR?j7l00`=S^XejR2g=E{7`uFvKM9aop_3gPm%<1k`rDvo@fibYEnt>)kfTnp!G zg~_djv#&*XfS8z*VbJ@C8(1`VxS=L{)PAObKBZfd9K9LDHD)YYbq)71JfaTpBIGzW z&LO6JA%vEKC+8lPxfYpF93umAX=E_|@3s@|e?;;B%ga>!&$iS5vzJ+hGdhMdJUP`? zIWS#1K0j8sfCB=}PcMzm_04u}*01dDVq=5qRu1-#HqSt(=iBN1->&mF7FO;*CocU% z!QSsY+0W83LkID6(2e35Mqtsu%>!dUE)T+&NO|R|wMpK@kTQZ(DFye0Tp85;m`6Qr z*&rvgL;2kVbn1CR-kHt^1oj$PGfpg+8^8j@ z_09~M7MRQxPPGb`>*C{?gvUFmjXIQ3i;65~%j?GN$###k>$->P13|aw1TGiamcx6$ z#^wDe_?Wt!zoNTHKDe$_W~vD8FRyuy&ZMw>FO!n+ww?7+Jv=?nA=>Y7RY}6A zaZmA(M>F*yuYO%z?5$#bcRem9-#6FGX?fBz+-S-1dk#bYdzax$1|Oyl*}qu*rW$kN zW9GRJ65T{20cM)HQ=lJChd+qlGq3~a$V~!O$tKOAoS8;M!E%hl#UZ~F)~3xc`>RTW zK>gKL!Gh67WeL6cYo&o(1%6g2BYqL@$#^E>?o_b|hDwkZCA2gxrUACLovdR)mczsv zf`r@Q_>t_T$wgCDiqOGtrv9OMU6qVl;Zw2r?p?BP$lZXP4q6(!9OYzcMvP-j7F`SD zWny@oHR%?oM+&$aVggaYk!#z>W$plplj>f56D;x^2>}ceg#V)hO-U+eolRwF?n!M! z2)M4g?boSJL(Oh(Y+dp7=~)X9Es(2jQ1$m&-Qv=3eV(r61d`Y9FOZAGVhjjhIc zY(3A-a+j8j`^Wg;Y@()J+tvZ)yxmlel;~p|+E#KnF{J%A)*c@XDT5_G{Q334Yh-6w za-@GT;dW1oP*WVmIQyGv>>-YoN%FE_^)C^#?bhE1-Y-d;e@+r58+x|JD0Kdkse`OY z#DrsSIXFT*Yw{1+TT>rX@I&n}zWp^idcAKhi;Es9_xKxwT64ZpV_2p*=W2KGNtieM zk;VcwX)lWh)QWJEG{B*%5+aq;2#UTh!o}_yqMUY$u9URCVy+&h?c|Smb+i1`yvsx+ z9iid}HXj>S?Acl%@*g(+dJim-|5qGg%%Ncf)XFpQ*JAoK$E_ce+K(};c*Js;eFb;> zk5aIDiVD@}+_XkkXz{xXsK4Etubx!AG;eWJSSo$y+#*lOSq|@xW0v0C7Z;$u)F@s2 zG+TS9$jzGiJv2_?&5Q4oxX%o9Qlq#KRD?^k?&n%CGzHmOF@6lvwQDRjm@UcRd=%S# zE3U*EjDP)Vn)MujKdBHwwK6y>d?k~W)x3?>A*1eatrb=$RDnhaB6BTl$(Y2g5}Qz& z^UD31f2>v|hC_)n>~j2^s&roRD;k*lqXJdGR+aSK)Ta=pCLEo$O3@RaX?G3uRP209 z{_B%Z(Gu+?okZ2&Siv;Sn|l%Gs+lbHQ~z617cMCV!{6z7w_A_NlpM?Z81eDXco$~) qJY$4*-jJ!BvYK-1zD1l~R@#n!&EI^Hv<|qfE`gEgcs~OHB>xwH>+8J$ diff --git a/docs/es/images/logo.svg b/docs/es/images/logo.svg deleted file mode 100644 index b5ab923ff65..00000000000 --- a/docs/es/images/logo.svg +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/docs/es/images/row-oriented.gif b/docs/es/images/row-oriented.gif deleted file mode 100644 index 41395b5693e973a41ea0827844c7fe546e3565fb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 39281 zcmd@61z1(rm61VK6_1f;uj(%s$NjdV{!y1QE%8NV?n zvea*{v(Gu--q-(seb-*DbqQn5@s8&m^U3?UpZA6Ea}G`=6J%theFy|#!vfey0PqJa z6hQJaaO)Dl@&KTx0Q|LpTqPjH9#CQfxEKMg_kfx(;I08cZGbXgz|Ib!!3ApUfI?G% z@CJ~=2*^ADc58vJ5I}?jsEP-M4S_K`pe7Gkumd_EKzlwgSqDsO0kyS&q9PC*3j}@! zGGhT)EMTq#%x40wB0#7w0OJJKVu4|PpfLpKmIY)40e*fU1$>GpV08}=5dq8}0U;1T zM->jpve`WKn8XyfL`09wL)Ms8`ww$3?V==1lTDD z3LwB20bt1)*ckv0hk%1#;CnN$;sfkf0f%G2!8c&13s{T+PL_e~0^oQa*qz z6r}S@GU6{upDGzEfq(NnHZxFC(NQxdRaVo`(&HuHtf(O;)l%gpf5$4$EN&pErl}=l zVW{@nLgI~zg^miRD*0nRQXVrdGhG8+HDe`GGu=;mMqFmRPJZR9E+}Pa7Fasr{1oZx=RtV{V|vB&BAg|HV*64g5F- z{80v6f`)2J#`=bD^z}boyv3KA`o{W3n)(K$f`W**MoJ^Dq@twg$psHqNE>H?jz_JbmB;fY;lcji_uZZCt<8<~wbhm7rNxE$x!IZNsmY1)vC)y?p}~RvZ+*Qz-LS6Cj`p_J zmgc6$hWfg%wKdgMl@;Y>r6t8hg$4O}xjETcnHlM6sVT`xi3#y>u`$t6krClxp&`LR zfdT%0zCKWIFHaA5H&+*DCr1Z+J6juTD@zM=GgFf<#zuyp4fOSNKj~;|X=xI5;{vySTc!dw6X!XqN1qGMv?;u8{+ zl2cOC(lau%vU76t@(T)!ic3n%$}1|Xs%vV$*3~yOHZ`}jwzYS3cEP%Pdi%ch50rl% z{%k(zYC3FgI$k|FJQHDTJiEHqY+~`*!eHIO*yOXdiPhrwLQ4~~nNEy?`*Jmo=%0sQ zz0e@$wRGGv5XIM#h!nVlrI6T?H=FKkLwJmF9SutSWn`v%7$rqA+RI@lb+_Qv%Q6n- zCks7&8b0Dy*FR&9)H8Gt8^n=f&LlMZ;PX=d7!zYRO2Fr|usX(|{u2M<#lGS|#G!8D z-GG&ix%)@{5^Y)5z*He+1X@?S@7oF-sgt6dNk7*$x9h@2uo`CLovpY8X)59@hp7)8 zDy~ivE%bc{m*x|?<|uw6YciczttKGdW0*vsg@tDW&(LSF5SRd*la%R zX8%I%ie6HfygNZiv`i}lyDPEjC-h0#pliu8sw-Ypa>kIw^>|Q+6Q|CZJ5(D4DSHz5$oxbT#=RLwBx#l5+;rD#t zbrQ*EtpEv~&#I`eG_`Tie0ZK5Z^wq`G;+F%2K_*h^u$?4OCZ_mSV9vG!Djg=Jp5e- zFF<`kYx$v4?bl`61U-uD@l__3oai8b99k}lpBR4^d-qeb-%QZ2vl z4mmQ$14eZ&bJF0#6~x?a9_s| zY0H_Jk<#%z>7 zO6$9RvKrVSak3sfQVrcG<*u{btUcH}VXa5=5!!B|p2Xex#+*}0+Vz;pnziS}YwMjs z$(*W(!$$hY`<}+QXTV{T8_esdG(qz0xF$8m>m;Whc6M6Y^6~88JFc?r*=c?L;|D*GQCN9=ZX|Mx6qSG zYXfuY#7T6~mb#~zxwn{3MX$&S2`(k&5zfvwXbD8Vu9%6~-}&;M!zIqEem8RmGr z5SgcxEZgpnjmdAs(LAHT@pFG08IMWdB)?4{^ps`C4{Ak8dHsIt zljr&VE|rv21Zl0}hW>zmbRi`T@362GlqJxcCy7$DOjugEH^{+B?Vehh$XiDL;P@-| z7%wkh;*VzuskyYkWO~%9G?E|G{79WyKTJ%O_F-s)YBFO4XRBH-OL$nKI_t+Y@h8eg z7(Huho5AKSdR@Jd!<5tH8$v!cD=jpTY1-)EL-mk#S}uAMO{!XGOvzefBI0P9&4U@P@=-eK`Eb&!hC*f zO?7jN_g>Ni$x*t?>RN=dy~@Rflt1V-Z9QbW(%DiH_Lgk`at^P1_lwO1Domo-5 z_rvJPQzgEfznGzGXM6FV#Xp$TTMX@2o90?)=?k)qeqK#Bt@+}uue2G_v!z$$H@mK9 zQ*tsCug9K?wV^*nG(Ce4D{W$r-hSX^KTC>x)X2-Y;~!8tM+I=SvD@qf8CA|R-sbF3 zVKfTri=1amIqLWly&K_Exkxi<-r>Aq7_Tz6MEAm?+n(4kY4&tMqWO@-a&b50Vb#hj zH1mN7o4vqyKbGVVjs`r5P4ZQ$)>r|qp*W~Xc82`2itX{}=$TRR==l6=>b&xU4NX^E zo@s1=webg^8Q?;4cMJ0|&&>6h)mp&X7ts4qdt2hrT~1;9DEoAtWb?4=A>ZyEqtz@n z({Zm6pRXTN_z1VeK@!Y*?}eM?_($B6qQU6Bq7dsfIVRH)+^7S>!5?em8z%+8*+z47 zMWYd})yy5+z4F^}iH9DABFtTx|*k?WZ;DN2yK7 z@Ez}_TK6{t_CxHB3omR4g&g?~9r-m)=HEHdEjr{pG~YJ0!#OmMZFYjV*w%_CUGJyX+_55j}S>{%$|S#0K68tPeI>RCDD zS$*hPi|eVsi)ph9AE#!#<;X!`sJz(ZN6zV-v>OD5(J#pwg zg%6$K0HAX+&;>K-QYdt#6uLG9-8h79;rr}pdSBu9-Zk?%4D~rK^*J5#IXm=$5cndS zd+!Z#w`)Q=+y~^Q(EVCy{91+l>NWkU-2AFj{c2kLN|*d9(EZDo{Kz@| zuM>FP6ZX8%>3NUSyQ<#ZJJo~v$iqe`AcH2r6+IxyEg(`eAfYATF=qfjr#n|!KsHAp z`}@GurGTu|K%rsx7X(3G_?{m_ZMO#P#1CEGrMbx6xBb*2_Zi>i-7}Y?+rbTNt}F3Q z%4s&hZD&0hXPIG_JgSgH7u#(0V0ri85M)PAO$SmMC*D%~2by2Lu{+rjgm!6!dI*Q| zgaTG$uAx)6jf9rCtlo#YYB`0Lg?6)tRt<&dG0PT11kvNmQM2!(wb;W+o5N!Tqas?Pri7yh--iyj zhH=wG@4JU@4TpJ|$8423Y^6oVHkm5AMT(V1eBz8P2@5KIZ`UzwgUY2|HEdoJM*X!l z2rE3c$vjRHJ)|`(0zEypKFti5Q0rQ_-E~^ZwdLrAwD2_ZFvhl+{-fvzZSj-B2@K^x z3=?evt3@Re=Y7lscCG}T@Pwtbc<%5-E|Elz@E8`EM6vWl$&rNZvhW`-oMqDD zE)m8z4=25sjl{;(SF*6}YK`b7uz#2COn@076CS63lEAc!^bNe)+4Fn+G(TiX>PQs?u4mt zR?_@1Q{z1x!;e#;BWZCf30mcG?`a}sS6tMG(n-pbR6HV#j!h{r!9Z(n8BToDBaZYq z{VQQcopzieZOV7)3|!Ncs}{+M0~t%N(jsYXBEubam+imFX4wo!dz7cW^+->^1kyuR zvJx$9dRw!cxU%MHv)9tICuP$~mQvD=v!`j324qvuhO-FR)BLGpOK3Cdf-~~8G9h=I zKhZ{F)8=B7XY8S6HE(Bln&wIhWg25<-j&V#Dx25FrGts(+C}TSD4RNmnY|~HbMGW= zOFP|nB%4krpFt;w4ZnaUBR_@q3mIMF{X01v+&N4cIUC_w6ru&x5ovs@`CJhNFWL(z zxwDtc(zDBx6**lBwDaV`gYLA)z4OfI5Q!&=h{ZX%eO0dLrd+ICg`J*zo~B%GZFrIP zXr6p)F4AF<&Ge^%CPjNHJ-cLGo&8Vw{kmhxC3YFF&FD&}O-k&!HLt1b-F61_gqL#E zxr?Rcicv(1{7!PDT8eM46yu69fDl{4@N-U(6n&dPcBT5F< zOKn9WB11m*Jw1!Zp<_T);o@@2#Ab0&}t*e~g)elv8!Ij-jmAGD2 zJ)A&emT4Q2Aw5 zo!F|%##7hIvXX$lYGAaHjHl|hRg*Jao%WSiUKI+^)dn7`^3056@zd(WlZH9&;QNua zPwC^HU#VxOFV`z;sAp|9UaP-+rNN56p>@BRim*lBqk)4bPwcc|^iK7w4n0er@(H;{ zg)fb_yc#E^+tfZarA9V=s%%sE)JA&xHS@z)mD9Gu$|l{@m-=JjpS_xe#d2M&TGU(W z3^SV}#4KLZ*ZXFs_B@2MxhB_PblSQymfKF>)j8H| z9@*PL-_@6y%y`-aI%NAwMkc=~Gfl&Q%y z)-C1kh^mp!^i>A^Q~A10qKSwf{Ze;l-q24jM@%%(R)_OUDOdGW`~V88qM9|M>Sc+> z^1UagPp6L-##ORL@^nU%7^a|=13j%>=r(;Xa>OiW=B4-0uuJ9^K* z(;L6R+KnVTL&C72M>|+?H#eIhSxX`rC3;W?tWQK2W~(Nd&&CD#W@F`MGe#FUp>y05 z1H5{(5SvlVsL`)iCM&aMs1z3VvSzS&#^m(IUTw?=WLJtRjQKInOXAE2Pt5ga%^zn@ z_lRFsgmNZkO$N&^3h>PU42uh=9p1#Ftk_HAYqOf!D}n2iCQ+F>jPW$kj(q3@5p=k? zD~LaO)fBq)D0`{MyZ_qwq6NcPNx&4&#BxIQ@}S;)zr^wzK43l?x(;GJE!-Q^(D8}r zaUO+L2gXG*?-B0I(YENt4u!QGqE#-w)#auQ4c?_(=xA+aH>E;XRduG-#v-h3hH!nx z45!?Ladw-y#$sc{40~OlaowtO&VOS4t;BSA_4d5Y#s}h^k*@Wiu14a?d19uONr@Ho zk4s;o1~#B0I3!y}$163uo2X1{%NzZj*_+MT6Xg><*?e1fPgWnUFB#x8A1llm6EE?{ ztUZFQdWlWGmYedV93c`|AI90BgYDgeZI5p3k>lTgg;?#D)ND@yJVe_Vl>Ewbx_TGu_skwi@MY8B+dt%Ic;DEbT|M9S6vN(TpK|A_1 zJ7G_L^C(9DSgB@zl<9cjY;R(6TOBs91zVpWcEUR=`X#pqeU92l zmb`(r9orSqZ+)H|zHEDh20QWDJW+68HD)@kVEkb=x!GKEfSNrQQL{Hoay(CRmZX1{ z0KOE-{VArigqSnFs^$EvvCzh^U#kC!+5R&&O(K0Zq68qV|iQ>L!J6ZCH{{x78eZ{+;j z9sMah@c*>*|GMzNpWo8|Pv8B&F8#kQJn)~X_J6&4;QvLV{Y(0PU3lPs>oEUym-BCw z{$Cd!0As+fExifJCZ{W;Z!Z*V&wm_INH13)LRTmtTKHtNK%^oY{F$&hC;fl$nco7; zuf_k?8~^|NiNJrtCjZGE!~UCZ`+xpTR-vHFAX>>z3JR{CUJmZwzF;%iBf!N!*fliF z8*C;A#e~GgL?gSN}NXQTf z>RWz010i}spO!?!)=yk9-KoYRS+Y4pnI`S)7BzzXs$z^0Ff}rZPU4Kwb#@+{(1iK@ zhkUS!-IFi2ruGgtRxY+~b`DxjULJZ-AAjdSPxoMBt4jhupM(UV-$fFFAc21%$jDET zehILPBz$pM?BX)`X#$8aWE3tJ3Oj!J*co%1a7Go!YG(zYNCQ>XdT#gGO?%Kn|Cavl zJI9jX*OvWqH7{PWcwO>r3KCsbL5>m^L;m#$2;>F0$IIXz(`y~zdvw2LcdqAS@$ znQRe>7By43W@09?C7Q-19eNd$TcuVKIis70l{T==6Tr^?S0~?03#ui29nr}TLW6`v z1cyb&1fps9XorI=g_QK`X&LER*-^=PnP6|fFuyPt1SBfU%K~ci8X}q^Tbiq5@!1ib z{2mf7U6T8lqhsR}^haSU5FbYwH>tn!ny>et;kpz?j>6`g^|(40q7pe>5>UH9d8oc4VY)VYy>zWo2!> zXLF00nsxu+@aUL&Xsc^y8Pc`7C^^Myi*}9ep^SGNt1C8!x+Pg`e4{7+Lxn&Hxl^+b zsi4OdjQH7>07~)r56gZIm%N$Tr4(E9DLW&Gb!QsVj22s>`2;Gxgx^43#5{hx^Ty+b z=IdngD86UmhAH2|U*;Xtw-wF2N*DgR(6Q2|Ht2e-l3u^v5lFM8vzub+>JV1h}?-* z+AZFdNe^Ge{-Lr=f+qu)nTk8*hD_`g`*UIm8Uz_`K%$_ffF=PN5a#c8C3ayzf7unn z+{FLkGJ>5V0V)${g_j>Gl7p7y|H?Q!GqiXMt9j*9+rInade=eGB*WqEP4_3a4b|sZ zjbN|7v8ge>?~h~Vj(56{2Cl zM`8-l_W6f|=?e9w6J`u$>L2?KEaYNrH;%qYYuYMXJ8Lo-dOR_qdI>1~Hm<_% zd#A>r2eRX@n5BqreVL3aFm#iY%?Q&JBU1AfL8HwzZoV5BEUM2e=Y(+TYx9%ZLzv~W z&Fm7AJ0m!ZW~}&Mr@(+HUR%tpviPqq2|@w1;vpm+PxxJXI8Ux8x-(3AT_#p~^ocyTX*!hX7X6Kuig!$tH8v;*ouZ|=)AerG=@dFj z^CpmeDMX`rhVnpdeyB6iVV?JhzHI6@_=KyB{7^Qg9~>cn_`qTUgg$%gxAbrz-~ znKQwCvcJ(;|AL9+%<^P!(H16oW@8DgAPGg;EnXWUu^lH6nzX{+>YhO1drC9ybVHhB z8l6HWb=u{catqHisLtrhF67eq%I>$K-N;>Ovs=jB8N<;*P44AT@gka_QTC?Waf|oC zLw+Cc%e_)c=Eu_HmgvNL>B8gxrJ3+Z^tsj%p-@!6yfiCOj!-%x5>2Jt6y>9=Un;cd z1if^Z1Kdk5w^i6M!FG>qZ0#T@mz`Z*Kwg-=gQ+*v(J#O^zzh;>B;x526^RlQ7Uv%p z9-3sGVwvEV3C;eJYm%Rr?r`VA*&?Kn1l1ZIcepvq$GR@JAmUCI8omRy6&bU`Omo(! z)oKH)XBzK8)>fOrL3v>&mmu`W5J|`m5(>B;3(eEE(XH0J^xxuG zv}SrXd_8)YvF@Z<_NEmKxjOb|6sGpQSO&Ca0P+g52e*Axs@i`H(`-nE@Gu>o2@he6 z5zg3}SAUOwaEt$sGE|D2&6hn1m{r;$g^j2@Ls*SQ8qx}=#ZCwqWpF-|)u+HBo(I%D zO@DL0FPSywiQ2U{OE05es)>k~f1w@7ls8qs$<3xd6shqg`PoVTqFsr>Y+KZw1BhdV z+3w1Cdr8h@jolAI4*n7Yc@+C|L!E85#JCg9Q_DDaZm4Ai8dcaV9>n4ohT2(Pw0dr+ z5rmE-zZohPT8mx&Z-&~PAR2=(R50wMRFgR%!VVfRU4OW{R23o#v*A72UzqeXWwPb@ z0aj?UFcMqf^z3B$HmTiN+KVYm?!5_(Hj{UmfnA@#2}~D%Cq-&R_l`3zh=?02U&RWIazK zshJ-jc&Sik58pt)fS^!MT58uQH`=f;|F}fYgruaDRNwRr=E&$gR`-m|khGGbtnxCS ziprwk)SCFJ##E;KHiyEm#a+ow84>L+)Un+U8oEb^$Lo9JbNih;C#C}@o5z+$)>qf2 zYO^=Jn2^CV^5paw4BW_=cZG6E1+GG{X?a2o;_B^gP=1eC@~NwLMQ75!68d?z2}+FZ za0TNDd1C=4qlw zJ&WTC3g4x99w2gn5vBc>tUvly1>a39woh*(C|hwTZ?8Uep)mkJdlog-b1C_j&{a8_F;@--&>)AQcvo| znS5ETPjLloqD~LT=R0k0GxA!0K4_TUwej7XvpYHbD~&{z`N)TaNF%9IryS0(Q*j-y zK1T#z&1vUT-Ki;;yCzH1uB0wBGj5c@nltV+$*D6Q#4NQlp3F@&vtEw|G-tgz7HA+S zG~btIefUw25|A;`wdVYI@!aQJ{Y4mCRsAGThgAcgNC@BamsW7U7bq*zniTxr;Yck+ z7`m()`Yv8LDV#5#ASps4+?+B}!9+O4haD2P7;Uw`OdazCeS0z1X@x*D&W+eZGv0x5 zB_$quhM|?{EFq$m@ymRISE0{CueO4rtUC^qAFM)U-abknvv3{XWT$`b-_D$AA#NS)siA*& z-3At=ZxwB(u^(Y;^W`NI&Kvx5J@fu^$OY%;FEOK@`co5dhRHWJyMwP7BoGa^)X6qA{HI;xt zr&lcH(p_`5pV_K8*eN?d*SEB{)^+lLzU8=BA9N-Vhzw|5-&BUz=;LeHVmO{KmKGIwW^}dC3I`kM=PO1m{vKQ$OhhgV{ErqF_l5%i z689uYo+_^^kPUD7tunCZ&J#hUOQLCMK65`xO+%WmSZBB3OEGX~s zsBEq7&+aNn=!N2?4w7{Y&P>g<52u%od4HQ!GB$HdhD?DTQ6AB>8Gn^XBaxs9#CY zb3RlTiD11w(LnH;ymc#z*a-EWyP{nWyb5@x9hcUZ@-jgrN~9>gKSQ*DrK+tcV=zat z-Xki#D6=O|voBq~y*O*MM2bB#LbN1%{I}!+1&dg$H1|HL%SCbluL9`3?Hv)xMP!)( zN>{Al$9?+pN6Wp*j}s=WELax@vY*vM8RS@X2a2^|15qdYOB1Q4&9^qfIn-w}{Ls*d zWjQxSvCSH6rVW1%2KGRK^eyM_NPJCQFX4c;>T~V_KsppHoq{;e{+k;=^`xx?PC||!#~u^;N@8)dB`<*;(ZB& zxkXq9(z!zLhx*Hc>JR4DAT;wI8dJw=;aY3}j6W$ANJOadOqIa9oXIL{05e+$<;hsR zq3#vp4%58Za&MByQ2O^w^KI;uq!S&|Po~;TnYeFxA3fCUwG&8@*1cWyanM$&hK={o z?vG*?i$vm&Viukw+eADc_#0?usqeLF>@R4h`o{@B!O~A?29;*Y?i`wd$J8rqOH;qc zR94dZ^O#EHL5d8<)Y3RNs$@514_au;j4NF=y0RBbxR8oBTe%yB7uSqgf)DS8nX)ha znM{Ho3wdkYRevUy_mqD3xLRkSQie-{q`!h21fl>c38+Gs?PAH`kc-q8coC_WRAH|k?Ms1p!VwsYu&Q9cXsI3fU>f5=PySN*=d3bnvoBH@d9Grqt41E1;ppoJJG0`TmapBh9iFRqW z8K(GTzl36ge1udZgyNXG2HwSY5-TX%t$J{?rtnj}@ zvFe_7B(S`@LeyrI))y`w!{<%=mbyP3*Nc4ILQ(x~hCqtT4~w9+p-8n(o6EGU41ERq zF)Kq46z%NFP1k3}GfMOouS?loymGRNx* zsY)({dO~bUu2&7wm7U0;W=if<(T7SNWZ9wQ9!%veF$T7iHEG-ix zzX_jDtYW`OUf$IF?k*cmduIo%tLJVDIG;H9?!v+lVk-ZqhsZ}4A#x$sKJGk3CjAv6 zMJZl@Arb|5zuh!XrOtx{SC2mI^IWs@H63;Cl6-|We*w!T(h;NO$`ra39P6V^?ZK?e zHC-7LtF1`C!$>7jk{5XIDf?&Z&c9$JOBAy|yBAA+86RP1C;R*7-3u^8f@LR~T|B&d zfqEDNmWg2Z;>R_&DF;l(7Dam;)FDMDWC>#3n*2w&+8U1}M)ySa$Mal3ntBdaEqK_c@ijDu=%aPV1bHC?AGu zpcJ97T97msyzEpqPYRLOGgl22vw5EsCKcbB5H6Yjo+|8FdFyFqlKn#Q8PYi)KM$FcBfMo}Kwsvs9b7*vMWNh-= z4LsOf_l>EkiKW$*)wSM@vGr}z+4%$Hp6$`yr61q7_FtX6^N!_q+=O60bpu;rR#$_T zueUIfK69OFcxFsBeYes6*5g}O?i;EK`;x5cTCwF(J`W)O$VM6Fo%|$(O@p!H{pZOx zO9n|7-(l}XF7Lz9ai+u{3End7(mfPqUf}wdEh02fw*pMDw5k=}puHCi$xsrZ3NRw2M%t z^Lz`1pT7pKOzU(neQRoLk57J3E7~FBw0yKbyym88gUNXK(GiEwOwkEL0-wz3wyY4D z9pNpvI7bpjd}V72a+!EnqGMBK=Ns8FG1&YK2n>TG;p&s2)LH0*nU+}}zB6=H=#v3_ zRX;XjcS?F65W}GKlOV)M3J{m#R14%)xUU){YUrLA!s75gF;q^wj3P|xxqGsgz@>8# z%;3D;tqr+>aMF~TAiPidds}&qCK80>wU(>#=}HWAC-*XAFcA6DP0;&-xr{cbc_N5R z|KFu~NdG&{L;81W-p7C#en(6q!W$ScYnBS4XNyR!Pz5R^qQeeu$t=ozSa2sg%$dVdadV<37aM3puN_=~YE&jq}Eet$mj zjkMN$kc@iTeDDX8<@pdrm-`E$s=-8h4>%ES`kw=#?!*W9gNT#kcE~Y*cgWRqk|E=vA#x$T3x& ztavh&oh7&DRHD?#85E-xMC8^Yc~%%yAM!*RS01*pWQY}XJeajI>LlCzWZ1=o(wX0V>v~jv&*PKR2fb{A z9fggT&`I~cy?Lj**Ds@ywKwp=bbW77(UoC;NHs)ve^@gmYk%ZZ$@>22YZS=AnDLl8p?7 zJ8o^qJv-P(fjzm1RtRt`|2#uOi?zc#k5&+z{|n*<*F(ff5MOEe8d_Rwn%X)+>PB;S zJ*=;xf1nL)5A=?Gn;09NnVxK)@17lA8Ca{`=-XOeYTcQd-2VzWIsJhExj@7{_A3J9 zYRQeOcfCp>5-F4iCvF@OVO(UJ2%!r=p}{m3&B{*lTxcPo{|liBU!4p6j^KYUbgi`g z@4#jDUol_sTp$^l{|kXcE}aX+Mofk{Potfbw6muJ*4y7E#MUs<_-tU{+r%{4)XenU zd|&rc3+eFa=B1|9<<-T#mEGCHp8bQP)*p$NGud8F@#$Z_MDqdu7M<~YPZJJuVmT=B zh_zK|;_9ASlSC0+!C*;zW`C6}{SAK4T#G(aV-zL+bwzO__NvapW!&*DRg8C-y2AOY z)iJ%ynAStv6jD)~i@fuRsDNEC=cVYhbCy;kYSKIzf zAUj|FN+7pg{<{RS+PZ#y13`be@Vxx?@C1^E)Q+)~8Kl1)zf*L$0w$1-sC+cBj(9ji zO3oMxZb~lK^f;7U?`VU}7h+nW8Ao#EP-SPjv%z>Lrg$@QhX*!GO5Q-yzcYah{XK!) zMJfWLIoN9alY|C=6oWbjUcSV9wN*GHNo2fJZDR_}Wkl0`!8v)nt$5zDDMzj^-_?h0 zG^Iwk`_bGwM`IX0r|{F?3c&M4K+8cjt_?;I7-+tIZoNSj(M|m*J(4hKfP!sw%&?+= zll62jxK)|jLVKaOU{h&~OCu+2FOTDKrSTb@W@b-M@8srg>*C=Jb`f2D9eo0vf`UE3 zlFUCcFgh|KJ}$;H$u}V^Jvh@YJ0LeL)jdBprr3`7qU1uDa4xu;bHW%31kvGjCU!eN zfyVinpWBVV^Al*C@uEDUex5)R>W~nZmJJPv2+k=iN{fliD*k6D(EQevtj0k!B^OGk zufC*vxv{mm{eAc93=D7iWNGyf7>P!Q`yBxXAudfBgx|s8HM0}ij%)D1zhm(-^x8}# z^yXuSH5x<8R(GoV!H>!wB-P{4$ZLCX8mfsz&}&(ITgKDqieWcb|1NyZ^SS{t z{Bm+rbKzK1baGUFNqlByO>IfVSFjr%hljwL+QGsEaavD5LC@gm5Li+4QrzpBgHcRP z4KJE?vH)DL%z-Gd@8e_QH3X8!wvD4qX7@qnSk7 zTdsk56)ByRuYoYKvoG1>_z!7@_uAbVxr@Cp94M*WA8>X(EeoLPh`ebjdt=3aNi>Gv zTm6m2gBi(K;uxwR%rB~~0WTe;Iz{}J-((Ax>=eG2qVJ7V_!2%}#`aXUND&w>#y+{Z z{=rjXws-zGIN_}*)$G2w`Czs}*?S%Jy8ETM(z%WYZN`;>hTjT8J*C)c%L*1@vAm05 zQH!7>B_s>T7OnPYeQakGr+Be68m|H~>9i=(A1SwmZ7@WdZcP?_rd#AGFV&u@i{ox4 z9XH?mHt4D0V{^J^vN^71E14x|ho~rCJdXW6uUp+Ms5vLsfe5bE@lDMMJM_O0+#Mvv zDUgl?c18a#uOkz2oALNPuNxvq!JI?nb)^YdjE6LHzMROybAHc>-BlfYMfO8g1Nitb z?)gc9UC{urJWUFEYgnch{LCgzHRQEFMp778{BUBpiU%iUgseJ73iLii36a-PM?XQ& z1M|9(VU1WP91+bp8{+hoIFF51%>;WsOszzBiE_=P0Fm(f$-$Uy%W>gP2(=RuCf=vV zr#Kwnk4;n1*2*yb)er^Mau=!W@BLKyzr#<#7x@?bRL8&Ir-=UWQx)fepjp}xo=0|K z;{SbQccMhln`;FH>ziAu8(TZTv$C3CbzQyn-}+lYPt`NpH$FNtJvGrb2b&pM?q980 z@7-KlY}uZi*sCG^Bh!FKf%9(8zpKXke+FG}Jx3Q-$)7mJf{F3^Q_>^ry!bOAhd z$@*kOyyI|vrpcQ`FqzKW(CH333rI~Y+x4yrn3Y!N8DUxKPTF&kt|HXauFTnOaD1W z3xl2cyv^sEqo-yE%Q4qeMG)x1FNOrXn=^sNZ~w~|fDDN^$dxw;U*M*MO=z+d=X}?O zM%j%VDx>U58J()+L2J{Z?8#WSH0?>-I~4EDJj0RTb!W#c4tf*LOr)AlpxZwRHzdk=`cAtI zY~@eomTEPIi)ZH{NRC^julYeLx>?1uE=H={U2aYNz`8Wh#!MgUV z`2>MfgR+2J1hg!>Tjz$43PC!D@gNs49t0H)9&}^eK_o9RB|IcGF*zeJD;o(sBrmrZMr~+gP;(6i;)uLXT9)o_{p~D$0|P@N<74Bn$*Jy{+0o&JiKXSP z`Gt+;t+k!Swa)KThl8v8$19tsXN_1wx~P|h;oblLqcD1?J=T&39(M;a+V2XpO#AZx zUX)zyAmUHnmj^29E%<0apfB|NkUvjUG(?Jp8uRmzzibHNkiSCQo8pKRtNP}R+J>6C z#+Is^7-HO=Ac)=#!zc$&MjRXwA08b6SZf@<(?KT`W zo*W-EN-F+Lv2yYL*g-(RAM(QW?myGDbvpeM*QSekTL+~;yr(1wB!5jJ$X_ZT`RfIQ zzBJ#x*oSOWx02IH6&AHabmY474>rhxY6*&jgHRuUr^!#q|58R||rF{$RQFy0*3wMC1mGTq&7u20dIs4c&kA-E`_ zY93ob&MyDfsE)P0^60NdwUmsYGd_!fk_*0G3z-wK4Ze~axvb0#=-QSPJ!sMoNj<6S zWa7N;_J+oJ6P6#wL+OIuX5DDdmK0H*-9W%s=84bH$!S3N>KO~hd?1+Ss0Bz}EpDqBk}{9|2fQsndAu%syc@*~P<_2AYNKiZ#&>dyh9 zub^hZ+g?=Y|2U`lURw_f4`xfQ?9_xxl&T{q|edC^5**Uov? z7cS}7bdtb}-;j$tkh*g+$R$Sja#@fu-G@LRn!-$J5S3eUdQ36&@7gV`J%RJn zbe#RyEz)jm9NZy}XjiA_ijpsAojKrStSO%~HLtFA9n7-UvR5>=bu=<_cC+wMAe8$3 zEKpwpvY#@5jPY~lh@e#QyCXtG(iKhkU`3bL`C!F-DmYm2;`hM{jnauy%D)+`p!`26 zRidF0A>rV(63D1Z^t}-q8=aZ~o;>QG9hsF!nvhh2?3Wi&kXi+fECtpCHDovC)Yi3x zb_ADpdz9Bze*M-J8{Y%POZiqmFxd!_QRk;7i$;C==7$z$Haq9SzPD}c@3tTHSFD!c zacLr`sE(-IX8bK%v2`w2v7~eNnBFhGAc7!Oj>o26Q5k!qivsXy`QZZG}H6RfPR^_6ah6gs~TtsxKC- z8Sk-#n}1TO3})FHee?g3QpKzQ`IAybT04P+7_2~4G8Zjr_+Z7amh{OkD$dXB#TECr z>;*-#3(Q_7x?*sB7_Efo|s-s7C3$ z0wyW?wEmqU?V9VX@_Yqp8>VV^vPWO*l*tkg9vqoV+S*x&JHB<5ad)$Siy`p4iNP0# zl?i40$YOGT5Cg}KA`OB>+SzGI3OZ0G&V5tO>}f}c5GyPaZ*I|LEn1s##*ny z>fGMc%+k^D;r@@og|lPlzIS>=m5!Tub+Ipe@7B6JAVxiH#*KgEK2Ix=$!0)N488Q= z_1gyq5?AmGC`(J@=j&u94sW-9Q~d110@O1J%Jp91QGv zk3b5ZGXxR%1&0TQMnnft+=C|gP{hPUrKFQ0#tB2h@-pKSOYnT*1BF!u71fm${&jgx z;Vlv1&SNMId*B09{qO;*-sgCVgJe!{&ZFb7vccFhkn{Lv!SyHS(N1q)eD(ACRBjm- zzH&s3*5uYTRXnDb^~|_GQn@)?NrK(3Fvz}^xjg$EdX*J(ti)yJ8V1p$)DNxTNsQxAd<}%v;V4bYIpo|oi?%b{CWvnnlM^5kH8hFw5K^;nc6p3 zMY11#ow#vFdQ}n&!FiNkM?yuK0SVY8jPj@k>nl|era6rC10`1^uo^HQ2@jQ6!;PYZNbgNN~BA8ePKE5~&A#Ky$V%)!pq#o1Bc-PqL{ zYT>KuZyMm>BdmR4|GVwO zT@*NifC~I4SdJYMFD9nuR%}Ti`6k0%jDnFn0e#BH?rHhLFu;q2Xytp_tE5+coc^?6 zXj_&d#+4O9u!Ylc;}ym|-qH=`NZs2q5ZOq+?k zx;v||xVmmj;}9GY+}$;J2tfiAPH=a3cP*?6cXxM}!rk31NP-hAxFpad-+%h~PoL8l z-F?@$`*J^f?zQHe?-+x>Ku=eb+j^;5-#VIQUDPtO#MHdFGu{1krVZsDf#lca#qzK@ z?0F}#>*WL-<$tXlB17^&3fTXsoFAx!|Ee58kN=~}G5oKUBXZIIk9L9Mb_VuUyKtW$ zC2%`p93!-ev>W*eUCKC4;uvHSFOB?X?a$d-nD|Ag)i_C5KCLiW!s$&xiWo4BCYe9{ z&0d08--zhH3E-}vC;m&h{e>3)Jbpch{ta4utyvhT$$wyGIFDMY45oC><)N?^DqAgpp+4lTs6}DLYrSgeqa+4LW$|R-Vv>xNEXqxmt zHNjK=sz9Ir*bbuoUu_2g|LnD({x7zJJ)izlb^iG!{D0IV2K<9KH13UJa<>1}2`hw2 z_=`AbL681H9P}2BW2||J;QGQzWST3{YA$Ew`;p)8DK*=^Y;0zFE80fFxkIqN4xpKy&8+Tk* z%Yn+np)49a`Y8WSYW+7bvwWx585+}n(V73LGW}@i|I{N&n#|<>Lu!#>2>!p3S_S@? zzjFT~4pcE8uP=6DKOeusHGtPqIB&ct>rp|x{4j^zA2-6^wF>z^#|r_$za1~6Vt!#>VMuX)Sy6aNXhl?QPJKcnd2>NEsH&l(ysNX2TJ7)gGNh_HIyp5k zK0Wse@lGs`EUb*KtxvxyZF+iS?q{f3;QJD`n9VT4IOXa?PzHN&Zr|X=?<|N0| z|M7qOd-0cmVuS|&+Y0M{4V-{h7Wo@Ep`65mmtR;=TvS@Zm{M64B+SY3XBO4SDO~*; zIFU$PFTm2<|LPCXPn-nz&oqsPlyG68a~TA{nfW>^uAo5pZE>G}BF z_HHzs?Cn>mS7F~1GC&T72U^lWha2sHm7*^2T(jE`nI4H)bo>Uhog)nI`r{s}5DN+52Kvh)gXLm9=q+Z} z_-!`?VA=C?fk*2_bY2=Cr(em6&^c~zi$agU9e4y(B;i1?^FfKs?NUHYftdRlseG<3 zT#Nf$scM4|=;B7gg0$hh!1n3McB4Z(1yTM}&qMUFIUXz!7rrf-j`>!NJ*N8DADdKKCN z?bNuWWla|T9`!JrRj%(bhH)Bs26*Yt>Y7CLjT$eqD1Iy3+LS}8lqw;40DU8`&Vf|p zSP=wcQzSlm!&o8Pp=hB8PlR;uGdXHK@vbyjZ6VT|e$otBp1KSvM=i@#hs>W1WKN?j z%U_*m0_n0u7WWc#)1hoigxx*54$OS!TWJq`kA(%aZIc)YXR_0#4$O5kjpz<@hacQa z17P->Ew~Yy0}P@`=5}`yFm45mUP)>V`A~6+RBL2;QF_BfKIn+mvaqs4B_$TOHwEw+50Cf-aOE zfN9y64IAls`wz4^bV={&ys|dZF3>MHDj8lnMWDU*%YWUN)8~VuX)*hw@=MCyzREFM z38SQ>&xoSAY8`mx%InmPi=08Gw^21=3xupP(E||(+gsK zC7hb_rK?50VA*bdNZ8Rq#*koUNoLqFzh|y-qqE)|g%8VvURkOyO>E&&wx(*lraUcfbPw1uw(W9MUn8C-Wi~9% zLt4UCyNQxJR3AyV^WAECEsGp}KdL_-^RmAmm7vZ66Q8hhXA2qb7@~tUCI#p)m1gAw zRxq=&0K6UQ&=wZ+npqR1DfMrZFy+zO4azSHTED7Vx#Pl7u=x#hd}~D(7R}H(F8#u_ zcSED&rM*_e_^z_#%e3$J@r=d9!{oC!GGQHugfgY@&DNtMoNHS8(hiB#BqUhi%Xz8T zF&g&syb5j0RkFpo7G9_}tcYc#%w!``AvTgJeK3kf8u5xiBE>`qa|X$|UJ(D3@K~rv zpCRhcLu4$cNX1WrB*+A=7(Yg2U!3Et@^($sd!=q+%(4q99C=FH(TB|A62HP@EccgV z!_0cjGrZ2Q@IMJCIf}}1#l7WrsW7Ig@OPO2;~1w6UfBM$4rba~nZc9D$&88&43sMpXXkEG%yd112irChv!b+}3R>uy6l6~fBc|DVm)5V$<78Z6 zgMlb{Gv-&GfSp`ebYW8_MG;?_5DjBWuOBw@Ir-9vvylXC9!`o!3cCQYYJizneOwk7 z!xgn5P}){8Fsbdj2pLo+Ys?5k@ecZpcy5oJyH;=(rAUcTCj)k9I2;9%Aum9#91)u` zn6$L9h+X$c)|fDWbo*VY`pYSUd=sNeKJHDaw5}CE``bcFxl1`mNPrdvJBUXV+~yH^+)iL`&OReeK&gY%a@VY9bKsY5`v+U9gq9sI*H>%1woZwItm%a$^!M ze6=@KorY?>OCD{A&C7|Ic+|4ftqEkjWRkOD8^zAJ9{bcH!|$mV{!@PAF+Rl+XX_fd zksIhcCZaVmFrD_RSv7Q-EfCQr`aTyyu*gtU!B$Z5BhEr1tzIx90w-2@9D*9Tu_(MO zA7NgUjX=uGXWaVQH@F@Tp$az`vZ2N= zR0&Isr6j+PlrkO{O68l%)hCbD+8-Bdqnj$t<$h?bKQ2M0nyTF=f9O3uF881}*M`ZR z7!f_KjL0|Fr@Ty_n0_b>dfI}*fb<~BUjT`IZ6hf_`tYVMLO=Za zh8Y7Hpp*X@uKnwuz=^4!(cYOmvFQA$^q*e?Cv@g$d0R`mQ;u2-zwXj2@au+?T&8ya z+JBesNrOLm#G-hJC=B(8;im|f{{mGFK(tbjBoEH}u(2y}H;?PZdYy~c*(Nn?kCpoD z5-*vx9EI*R0~P{enS4B2JoREFj|=`-(u&A@2%SG!2^7oGJabxppPJC)-lYwm^h$pE z`oqutg=|-MQt!+g+iSy(tzY-TM5yu04|&hA49&AwHU7_BV$;vu!Eb`Le}vkO8{2r~ z@q21Z=rhyU@;Rix)t229eo;Fnp6{WC)N(rAzE49lt&h-w!6V~I=Wyw|LqzV} z^-f0R)=s^0-{RPPl5z9rRAUGKpz2|W?kc>Rw79h?j@~J+>h+?UBaQmepuV)Lx(g){ z)V_BTtlugg5oF$%s@a%u#cIu+b0S3x!fVLxX4vx9h)~U>+x-hPnlAF#|FaINld3Q|D=g|LUHp;_o-+y~gN=-+-0 zLJ%X$KL8R~DE5^EXNhYRxChhhTYVV`q6N`I3ccssot%5=s~W3FMZ^$sVX#*zrlT|$w*{9f<7{zzV%*!a8a1?NLU@k%a^yN zU|tYdK?Ag-p-JlDQ5FG79EI1KwS3rx8W&!zVv8xlZR3YJ9B$V)Yf z5|0r=@{MW;r&bLHAI4l9BG=Q$b}WK7@4#=tR;{VAsIH=2sY+W-e#npjWz|Rl*GNNr zl`;G<#FfYk@%T&Acomu`+YwbPjer$=>otuq)RicY`=~3rK&eH(U5~(OK2UE73eJNR zo~18JLtM!ch*T=%78EW504EK`9MH$L0OFw2L}1fE&5*>hP$MRQPK<{VYFg4duP7!1 zsK-3^7@{o;NQNf}QlyXBX-V$%2qzQ{qh?UG;!~z4@Mn_JW%2ZHcYFz*2BdP8LD|p; z@JR(YKoV+~60|MCgf&$?`4YuwqTz?UUoZ`bZ)0RFL3KBYVEiQg(KsJX88QY#HVIC~ zLg^%n^g_f4DE^>KKx|z}5K>t(G69!5f$!~Z>?2mn@6tqFf4PE`*YLn3j=$l7#q9Z&iR|oU zb>$URui=4Vwe{^Bjgao1Xo|n#frI4aL%oo(k(tr)k;%m+(ERM$^x*Pl^V+w$?;Be& zUti-H_fMKukB)~=ui`sy3v2G`uR|Ul8(yS;65RvTdz7Btuyg>229?3+3Z3bQKlKNL z-Vm^I9bQnfMO|;(dedFv3t08Y_bF?}@3F_80K#3Q(Dz&7XDq%i5K!VzTYT!DZG4d1 zmoDH2Yej&@j6atY0!22s7qgX%midep9x<%Bi?Lfdvlvw?RT~}!y_!OAndh5&UmPe= z(xXch3Ppd$6Cv{!eKmCXa@%>ctYP5jrhE+F#>dv!+fDfX8RY?2BOnE#(@_MmF10!*aJ7^$|CgWZ9+cG<~fiPtu^fBJ; zU8<&yByW#u$Nk*gJ6!N@KfsGj1A2}=3=W{Jj| zX$RR-$)kI0Mr%i~n4W8+C5aq1-)Q^;meS1A%!?V061#&z1%_TpD|29EAURSn)WnF< zTRDhEc^S(i{c&inR$l=^K4swNh>~dOic%pVjpdMT`g^#_;xTEfw;y7)PHGFMN)4Ej z5mKqZ28zvc8$^SaVDq$>=M7U_|OgpUozGOx*`>KMLsgwJM|TLsiaXf5}6QAm~RS3OdMP z2^1wf8lg@REY49l^VaOX?*z+q+fGSc^o4u}7f*Uifa~+zZZofvgeM{`ay`+tEaHWB zG7Dz`WLb5IE>1p*1&TseW}_O$EcN{{Tt9A%XYB-7!gG+npLQxNc(F#2mR8-ithJSv zxAgRF7_}BG9p6?7b=kIjJ$@FTXe6Y){}CxSqJ94TmM5DJI~sN~x8YO(G4nlaIKZ|m zl&Z`9>(^%%f$Bj2ACMi75|WnVfSBjgub)V#SUJwC@!58-Nxe^wVy@)N%NvOL_hSuP znYA(j=wBU%aG1(222{mP1oJ*`MK)mw#h$p#%RQ|M4$#ZwT)wbS=x5zLn4^aarKuA! z-?;xqv+rPGN69o<;M)4?k}j#;`SS82vo_m-CtQ++;Y}Oy?hbRP@%hb+BHS*)tMI>@ zrO`;*4SV&mzL{HUL)K8*TM4ymKVy7D^fNy8o2D_`a%aqwn&BUG-}!U#V_yyhHw%Aj*skOENRtk& zEcnu*!`nvbH|sdK7D(v6B1UHP!`7_O@TjZlbAMi`Xyx|osaI+{odlLEM2x8N9W}Wr z{$(skwvgg|EVQ`5W!#tnHFXe$6hNRpKG~%Z5q~vL-mCsatp1W3TZn%Q9ZdwO2#$u~ z)2Ossc70M-ViL1C!#E|mAskh15vyw1n0j%%cCAYhCQxbuz333H>N5~iv25Io$2n~( zk%r53bWBXEj$Ok=a-Xk>e2Rs`UaV5}a71H>f34QhQilp7YHvuncQ$i3k_u9Km`mcx zVJml1aQ?gkVE|TyC$}%SrD8RwIVi&}#%H1QruwqM1wQR!zcM+@Hs z6s*`H$CO@Tv)jZJ$ArlxHuUz8AvkJsBfrtD`_6SL!l z9Yi%{R=1Q^36HNDyxwIBBAC?L`LtV{`DlA8= zpJ_6xX7cyXJQN)AnId;ij($)|Gvz9&l;W~j;m;SXMmi;ClSzO7=#97ND~}Va>Wd{T zL$7EgjLW}XX~~ONw!p^eqK(c{r7*p($qh`-!Xv4&^iE%hM*m*$PN|AS%IZr3>lCIGZ!@^FPK@o_ zmRp})X1rJqR5KpG7QHr!k~7C{-2zMX^6zUbZLJ~OYZHEIZngd_HWU`^b6!GjExw&6 zW@K$JU|Q}bc-vE3g~zQ;Qb@}(v7K6g*0eagd)a)|nM%#tA~_jX2g_?W=v_2Ve+R_N zp13q^(?NpxJRs?zJE!=6L*c6NoN}MK%DD1LgP^v}c(VgP-$9`oAbI77kUD3r6CZqv zX~Xnky-CS=IL=b@WK!I8N{QLtmma2>jhMQw59&M-{p3BP8||u)@oUKxp)W1w_Ya

)tVttp`^7_wBmx@aUR{6nxL3BQiH?(D^G@ zN$kT@%9iZI-k_!q6v_{0Eki4GY&_}NfDY2uBZlwm)8vkWbJ{K(_j#Kcs@ymAwZ0E$ z-&K=rA$9x) z);f;=DwA|g$O!D_B8j`-KHJM-8!stRZcW$otLxmede*+mo&P7ifbj+YtbJ32Qh3H? zOBkd8MY;g>I}^cu{j#?0b(VJYe+&@(*`kZ3D02J6~*fFJY)R z)EYq9c;5p&p8?5tE6bKp2QL;G>i*P{zLfVtHip1EH^=b-U`dG|ZK`j^5HQP)a7CTA z{my(o%wJd|g#0IvM+BI62h;(DsNIJc;D;LVg_>xDnt6m;q=s5SLTy$;?e0Sz@WY(= z!dx`|6&~mX3G-R`6CN0V|2H{M0~G230;Ph&Uy}n@{w4?FgX8(Yi5lQ!4{&NKI2{7c zTmfg_gLCo2^ZCLHHNuNM!b?-b%OT;FE8*4m;kEb?^?d)E@W7Q&h6DIF76Ho+kz=I+ z@9s&5Adzx>kyHSn7*dFTGf)xcx9+03OWzd?N1Z@ysP1g{OCxmXgB9t0YN#j<%7o!?NgFNTzjo4QAYnRl?dZL4`AXxZCPDjM^R-Q5WCgNMDIly_;f zW3stjDT2Q$LyGzXNQ}?@@iwVv$je|g`aPd_VAzX8kh=wBIVsUCQ70`?njkL4AO*k0 zDon!Diy>)3(w~UmC-4Y_Q0m0&ZidI_tjC+6AQ@lDAAkXICIq;NySqryxfw49%9q49 zNX4@}IJ5BwRvHAnZw)jw&oFMu;2I5V0%yvp2hhI(2k|F6NhF6IC4tL;OPa}FG%XOr zl4Hw~;|a_YkGx@)v(iJ;l5^AI!#uNEeUc6wm0MFX&HZ2KQys&C;J}V63~8PzSxFDc z4iD~IM>aFJsVM6Hd#&D>lDP<@X1Db26e%k{uJDFAOUht&O;ihR!%N|`h#Wq@33?_ z&rHsP^e(A*RlH2$lEBaAE`So3iB*@n)k695BFxnE0S4qD2IrZwOe6JTo}zZ;8f=FR5mRK4n_}T$pRa2cC~u!ahm0c=9@E3JE5_>t5&ReYspELe*V&q0i%hz z4teMN7DS9W#9^h!xcP#IF_BhfU4&)&w-tSiuif-p8R5!4ud*j8q!fun#&BC-uL3{9 zayKh0POoy*mSpsja-XqsX{q$uwwD5h)q;7m(v~$JQNqG@tEAL1B(aO?i^pn>0QjF; zHMd?hKLu*;AFELeYaZKb?$hC5GT=1G;iY93~{77RDL{Hlh}_-Gp%6gl*hJ=GKUfLguLAl z@k8zTqwXBG9$lg~ql#YFcm4w$vExOm%p_imYt3_U6;&Gjba8#PB=R}cMo%g>T2tMX zJ^gBK<{NQ-_99~83~^BzJ)S}BtwbIJYWd!zM#)9J6B8L96-EoHrirG;`aNB?FMMt) zyoK@lD~nBm>-tRa{2O~pi48B}jWo*NZyGGxwf9{$?WyFCk{a*(^b}MeSs|<*6<@^P z30`iuXpA>&WynA7HkBlNX$WC3EdprYw1M8WgW%NaQD7RG)XbCAy02w=7SuXX)C1u) z0vDyn8`VZC)C&gGN3~^NGhc_U+j?VV#|C7>iZqrg)Ux5;2r==PR>w?fl&{63#si2iO|t$*AI)}nHlVazEMh3Qk2ZoM%tr5Hf>B? zS4>4=MxyBsw?Mr)uTS4_2xY(AOPj$!Ldgrb5z@&9g*_Yzb4o+-ui5M6&Xs@pr zNNa2#DD3Rc=qTB39~?j)aOucRlvX>D%G`yPl2tS1?yRI{0^`U52bpO)hJZgIn?$Yf zI+`}#jTi4&^Mw0PtovQwb*~IC6|VDZMtoR;Z&nk>xA75&7F3R9>PmF&>2lR1Q>9Yp z9__*G?Mxc3fH9zoq+(a?mWG2=4tA5_z))2VMOpX3bTBdaC|^`C6%nI)Oz`mJBW0cp z-!}H;MSNZ~7-(P`>NpvY{naLKBc-U*5mbcLYx1UxS*`zF zE@t;JQ@^UlxZu|pmBJI8jflQ2B2Dv%$;S7gnrUT`^C8%$vgY zoqi`b8RI&;+%ZC(HHAeoxAk@MDY6}&WrBrd!qjypg=@xSWiFX|zG;1mylU3BV#L03 z*2H3(CT`|~?%dw?Jb~>3k?))!3HoQ>G4aj?{NGdb8$$;43xvd>Op|lklZ#WoQ245* zyJhE%E#`Gi2Na@4pll}1))zpIB(hZtKXT_4b(gON7g2NjEOEJL%_DCz+t^#L17>YsK2D`ANc$=po4-u@h=yU1C8QH zi$f!894lMf<)aH5d+X!N`@8#t?BBkv{n$IdzKN}BZhQ=WcnU9n_IlVcA0=hg`;CCi zIr8eS8G=MX3xmUMDDI1<*@=x*r#=)@KjdhvE=MY5nZ6K3Z5dB8VnL&xZSzpSPZvjL zydO#DY(DPvNfoWJ>3{7|x?2=hazP(mWSRxqC)}xEpp9QKMst*0Ly8}s|+FXw#3R*Xp)B$+bjooxZ z#fX$*p`w4C_P_F79n2(sh>`D{xjvqYF}APv{orgdUE?jun(cl4EobixL0;?Q(OSc* z0z9tH>FM$zNt?ZX`X|k`A89zbG#4M6)+At&HMZ0r;NRxhJ%54ERfZP=CEFv(@2z>E z0hM*!cR~sU~K=W)cnbNS4J&B&Nq2048zySF24IjssKop;vg96t zq}43LY00t#+h((jF5Bho$Sm8h?~O$+P(E)wD+pWQAWPGv?m$M9t8t((K}iJuAIp*? zy|Guz5>`XY(!%g#%d*ty@bcpF>M=%~`kvasSIZLV3S2E4=%9+k=C8It%L*>kv+b|pzwOSwJ^tCO<%v-R_mb*J%~o)?5jmdd)?4`=P#vDU2p1K-z~x{(vsFJ3zF zGAb&om#JlH`mdL~G})7rC*bjDN-8hMn6qJuZACgyah*?GjzYE> zuV=)(zroJ}=vW$OwGuPA=A|WdUD&0?Sh%Zu?w@Lww4HS?de(q)7fTzozgYUw*(y2K z@OcGKI;^!PZu{|DY&p7m+ML6Ouh*V_ZrSoywJ_?r+d4FcC8R?df3u>q_f^nm{_MXk zN8{bF?*7OR`SgqYb1TJk=S9cakgr`=`~^w%RzOQN|L`{C*keUE!>VefF#)ZWF+S`o zJ5@3w#?Orm*q^YNInI7Qkn!#hK1U9cY@EkYrFv|KRj^ZSLksJ>?JwKD{u1+_^80m;HyJX+P?7%ou>}4Z>r@sh7x&DhrANxs z$VUGvxymP{IZ~I^LmMi4E%ktZ#x+`R^8MykbW7)KTCHKP*G2~qQvJzqTkq9=JQ%Xn zc7-?n4(&4H=QC>jD9F=x=_t!oW3C;TgpR@JPvlnv;=S+ot$^gaUw`S8emwc|Q?H+J z71cP9p-mAP|3jrVMwmJ;xlo-))>Q^}l>0zTTr9}0(5QXNgMX^d(JxE80L9Ql9H-AN zB7ke;33cU@==X3x*o7^MtCkUQPBec}<=x-;`ynqF9{kbh4e>Q z5@SgyLHQZRBuOcPJQ9vz$3}G|@k3EnQAcoAzcE(Lil|<*VrT&>Esr$6`_nB51e zZ2|f|o;8l-*0~+#7qfAL&N*jMgB`Al!|_E!jMP3$ee!TTM+F?huX466$mri_^aNr@#l{bqP@{Nt2jkAg2QBO#@G%Gj zIj;ktCg}k)uBdU7j3Bpm?RC~-dHXr+WDeN_xFA<1)95MJr;{CCb8fzyv22ny(kFDFY&}OpR82mshdj3 ze$dI~NiWsgiK0BT9?ndREj72@R5?Ud=p0Bdx2@e&drVg7-Hk1GJ>Jv=pj8?m%B=Je z-qwQTDvj{QSB3;`>tdrSO~@juN41h!>&K3$`LrsQ`ME8)7QdU4XjfK-s#U0>Rav%J zsaA)}hUwc{s!F!6lx9ujBZeG43wnlc7$r8lv9RDMYA-H?SGZt|9|(F&t4#ZqYuNuV z!!U>cnjy^{p@L)SxPu_4p5N%YNX>|>8_^Qol~^W|WrtAFE?#z-=rZ?>8 z8b7l;|AeP_-}y9w&4!hI30EUsHQK9Vc2%gdYaBUAV1QQfnn__j?W!+xa^tYUZ6XkV2( zu|DQmXL0sfH2NsN`BsIZtqmXDXMSbV{J7~}E41rc%EugMSaXPj^yH@Qy!mZT+E{G! z&-hs?D{do$(|nfGb!Z*;x-kaHoedlK`I8j4hTL{t16ahdjLBUmZN`~1Mg@EkSfaZ4*xz~zB$A6ndWO&e7` zV!h*^oFuu1NMhot__r?Fre3S4c81End??L+zJ6)t^*-i_@oE*haqAxMyzp-Eo~B`Q z?V!uPj5zq9({$#V?fPtPa{yTpQ*fKh=-jdR)VVC9cRTdrvlrX#)eV2{F?dyOGt=d> zVXDBmOhkGZQ}XfLp6%D#*2lXbw&x#V)AwC$WcPtNANMIdJjUtt&nK1zb;5dnQy4&_ zY+!WX4=ebc`FH<1ROo)_t$Du9>HfXr|LNDx^z%ak8UArf_wV)^|I53VXD3%GS!@a= zC+g-|YEwrV6<~nrcN9mO07O85@f_9VH!A!*GBo@^BO_WD_&^*G_1k+I9ZKq2VT=wA zS|T8l9}X3}6D8FgaQ7#Wnv-f)B#1MgL~{;!K_ZeD(_ zotc?ITEVMs@s9@!u&AW0uDRtk5-O*mwEYhW59w@r^??{2iy9oCnx5&OoPBk5te9VZ z4bg~MZ8NvBO1EOgCL7u~Uf6;dVB_-d#4{kkz;M(2cseQWYBTUah0tNl5#ZsTeVZSy z`Bv{%x3H@Sa3C#khsC*s+Nq_gx3{9+EfuLZi6R!HeicSJnTBJt8IK{Qq^P%dKN}-!(aK|`i4t13X?ZAZL^{D#?8ZlrqbiZ{pcT#X~oDRN^ycrdTkm? zgovs_^l4LVJ1&RX%?odo9#dLgP_YpR^AL#a2n;P?U^DF83`el`>wnO;ogbH-7G14< z;CLCF>l;7qD*sF%K?sxZWvEmyz@dmrhdA5<^~2^IX-!LCW_ESM*e|Opo`kT9sjkb6 zVhVC^Ed(gW;aoT>ol0$*H~KUM9L-5{-sgk&2!~fx6e-YfeHBAsBpfvDBUiDIZ|(UD zYie@b#U$3pNF~|zv#gDwi5c_*268YrLkjaAfCb_hY#IuRmN=_-#;^|T`E2x=cdRNp z4L=gpKf&-4Sa)Fcq3DEdv*M{yk6rE*UcJ937ja)hcw=mWc{?W^CM=3uv|K%P1k~sa zEwKhs0LHTDG9ae|>OoJ5_LUS|6xAi+E!0pkI=`-KGdF#Z+TY3`9fEnPrH*s+zc9bt z>?VA_rBtqb9cb)u+MgXJ24h{;&ZXFQ(|+;F<9eeR^9htkZb3*1v-in=l1=9u)An!N zR1kexr8LJ#-`>cK5a5@Lg$4ZzIW0)oP&GD?oH;lO7qOQD3d#pA0VVkH?`*hsE11tW5>21OlLD$>QxOc3~9qsNSI0Dei* zA4Dlq0oN<02ybJON0^%1mfFXd9!GG%1cy{lFD&A40luy<$PHE~o*VQ8Ra0q-oiU4W5!FRrg+3k*#B3 zJPgfR^22!ekxXh5(Ez>y6b!xr*rUV6jECiR!b~e7ZZ>ms+=8d!1~Qk)?`=Ba;yOj` zTar9L9M1ccJYjj`B)d^a$Nm$4Dtf{>$9vqI9r|&aQ*}BHo#BudZ*4RbTO}u-Us*8B z>AjOYdKNB;xkNV)wOjF9d`Nq!DD2p%ZMqZ|YE}ssrZf)-cfsb~XO>E-eA;B94>gr9 zNZx&HKKt>yL>R4HF^o_JDpDnli-ew!NL;CqwU>aGJ4B^Kda+#Vrd-2(vRtjjE7VoX z#r!llhEXeRN`uh8!a}Y>YdKw&m$)e#Un~}!jx1;WIjQQ;MODr*v}LJ@&oRn^Ieu)~ v7W3xsMK#PyB?Z?F%i(tgi$%JO=~vWuiP{KWj6`T@lyBoQT_wn&pg#N$h3a7L diff --git a/docs/es/index.md b/docs/es/index.md deleted file mode 100644 index c76fe32e33b..00000000000 --- a/docs/es/index.md +++ /dev/null @@ -1,97 +0,0 @@ ---- -machine_translated: false -machine_translated_rev: -toc_priority: 0 -toc_title: "Descripción" ---- - -# ¿Qué es ClickHouse? {#what-is-clickhouse} - -ClickHouse es un sistema de gestión de bases de datos (DBMS), orientado a columnas, para el procesamiento analítico de consultas en línea (OLAP). - -En un DBMS “normal”, orientado a filas, los datos se almacenan en este orden: - -| Fila | Argumento | JavaEnable | Titular | GoodEvent | EventTime | -|------|-------------|------------|---------------------------|-----------|---------------------| -| #0 | 89354350662 | 1 | Relaciones con inversores | 1 | 2016-05-18 05:19:20 | -| #1 | 90329509958 | 0 | Contáctenos | 1 | 2016-05-18 08:10:20 | -| #2 | 89953706054 | 1 | Mision | 1 | 2016-05-18 07:38:00 | -| #N | … | … | … | … | … | - -En otras palabras, todos los valores relacionados con una fila se almacenan físicamente uno junto al otro. - -Ejemplos de un DBMS orientado a filas son MySQL, Postgres y MS SQL Server. - -En un DBMS orientado a columnas, los datos se almacenan así: - -| Fila: | #0 | #1 | #2 | #N | -|-------------|---------------------------|---------------------|---------------------|-----| -| Argumento: | 89354350662 | 90329509958 | 89953706054 | … | -| JavaEnable: | 1 | 0 | 1 | … | -| Titular: | Relaciones con inversores | Contáctenos | Mision | … | -| GoodEvent: | 1 | 1 | 1 | … | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | - -Estos ejemplos solo muestran el orden en el que se organizan los datos. Los valores de diferentes columnas se almacenan por separado y los datos de la misma columna se almacenan juntos. - -Ejemplos de un DBMS orientado a columnas: Vertica, Paraccel (Actian Matrix y Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise y Actian Vector), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid y kdb+. - -Los diferentes modos de ordenar los datos al guardarlos se adecúan mejor a diferentes escenarios. El escenario de acceso a los datos se refiere a qué consultas se hacen, con qué frecuencia y en qué proporción; cuántos datos se leen para cada tipo de consulta - filas, columnas y bytes; la relación entre lectura y actualización de datos; el tamaño de trabajo de los datos y qué tan localmente son usados; si se usan transacciones y qué tan aisladas están;requerimientos de replicación de los datos y de integridad lógica, requerimientos de latencia y caudal (throughput) para cada tipo de consulta, y cosas por el estilo. - -Cuanto mayor sea la carga en el sistema, más importante es personalizar el sistema configurado para que coincida con los requisitos del escenario de uso, y más fino será esta personalización. No existe un sistema que sea igualmente adecuado para escenarios significativamente diferentes. Si un sistema es adaptable a un amplio conjunto de escenarios, bajo una carga alta, el sistema manejará todos los escenarios igualmente mal, o funcionará bien para solo uno o algunos de los escenarios posibles. - -## Propiedades clave del escenario OLAP {#key-properties-of-olap-scenario} - -- La gran mayoría de las solicitudes son para acceso de lectura. -- Los datos se actualizan en lotes bastante grandes (\> 1000 filas), no por filas individuales; o no se actualiza en absoluto. -- Los datos se agregan a la base de datos pero no se modifican. -- Para las lecturas, se extrae un número bastante grande de filas de la base de datos, pero solo un pequeño subconjunto de columnas. -- Las tablas son “wide,” lo que significa que contienen un gran número de columnas. -- Las consultas son relativamente raras (generalmente cientos de consultas por servidor o menos por segundo). -- Para consultas simples, se permiten latencias de alrededor de 50 ms. -- Los valores de columna son bastante pequeños: números y cadenas cortas (por ejemplo, 60 bytes por URL). -- Requiere un alto rendimiento al procesar una sola consulta (hasta miles de millones de filas por segundo por servidor). -- Las transacciones no son necesarias. -- Bajos requisitos para la coherencia de los datos. -- Hay una tabla grande por consulta. Todas las mesas son pequeñas, excepto una. -- Un resultado de consulta es significativamente menor que los datos de origen. En otras palabras, los datos se filtran o se agregan, por lo que el resultado se ajusta a la RAM de un solo servidor. - -Es fácil ver que el escenario OLAP es muy diferente de otros escenarios populares (como el acceso OLTP o Key-Value). Por lo tanto, no tiene sentido intentar usar OLTP o una base de datos de valor clave para procesar consultas analíticas si desea obtener un rendimiento decente. Por ejemplo, si intenta usar MongoDB o Redis para análisis, obtendrá un rendimiento muy bajo en comparación con las bases de datos OLAP. - -## Por qué las bases de datos orientadas a columnas funcionan mejor en el escenario OLAP {#why-column-oriented-databases-work-better-in-the-olap-scenario} - -Las bases de datos orientadas a columnas son más adecuadas para los escenarios OLAP: son al menos 100 veces más rápidas en el procesamiento de la mayoría de las consultas. Las razones se explican en detalle a continuación, pero el hecho es más fácil de demostrar visualmente: - -**DBMS orientado a filas** - -![Row-oriented](images/row-oriented.gif#) - -**DBMS orientado a columnas** - -![Column-oriented](images/column-oriented.gif#) - -Ver la diferencia? - -### Entrada/salida {#inputoutput} - -1. Para una consulta analítica, solo es necesario leer un pequeño número de columnas de tabla. En una base de datos orientada a columnas, puede leer solo los datos que necesita. Por ejemplo, si necesita 5 columnas de 100, puede esperar una reducción de 20 veces en E/S. -2. Dado que los datos se leen en paquetes, es más fácil de comprimir. Los datos en columnas también son más fáciles de comprimir. Esto reduce aún más el volumen de E/S. -3. Debido a la reducción de E / S, más datos se ajustan a la memoria caché del sistema. - -Por ejemplo, la consulta “count the number of records for each advertising platform” requiere leer uno “advertising platform ID” columna, que ocupa 1 byte sin comprimir. Si la mayor parte del tráfico no proviene de plataformas publicitarias, puede esperar al menos una compresión de 10 veces de esta columna. Cuando se utiliza un algoritmo de compresión rápida, la descompresión de datos es posible a una velocidad de al menos varios gigabytes de datos sin comprimir por segundo. En otras palabras, esta consulta se puede procesar a una velocidad de aproximadamente varios miles de millones de filas por segundo en un único servidor. Esta velocidad se logra realmente en la práctica. - -### CPU {#cpu} - -Dado que la ejecución de una consulta requiere procesar un gran número de filas, ayuda enviar todas las operaciones para vectores completos en lugar de para filas separadas, o implementar el motor de consultas para que casi no haya costo de envío. Si no hace esto, con cualquier subsistema de disco medio decente, el intérprete de consultas inevitablemente detiene la CPU. Tiene sentido almacenar datos en columnas y procesarlos, cuando sea posible, por columnas. - -Hay dos formas de hacer esto: - -1. Un vector motor. Todas las operaciones se escriben para vectores, en lugar de para valores separados. Esto significa que no necesita llamar a las operaciones con mucha frecuencia, y los costos de envío son insignificantes. El código de operación contiene un ciclo interno optimizado. - -2. Generación de código. El código generado para la consulta tiene todas las llamadas indirectas. - -Esto no se hace en “normal” bases de datos, porque no tiene sentido cuando se ejecutan consultas simples. Sin embargo, hay excepciones. Por ejemplo, MemSQL utiliza la generación de código para reducir la latencia al procesar consultas SQL. (A modo de comparación, los DBMS analíticos requieren la optimización del rendimiento, no la latencia.) - -Tenga en cuenta que para la eficiencia de la CPU, el lenguaje de consulta debe ser declarativo (SQL o MDX), o al menos un vector (J, K). La consulta solo debe contener bucles implícitos, lo que permite la optimización. - -{## [Artículo Original](https://clickhouse.tech/docs/en/) ##} diff --git a/docs/es/interfaces/cli.md b/docs/es/interfaces/cli.md deleted file mode 100644 index 395f9831a4e..00000000000 --- a/docs/es/interfaces/cli.md +++ /dev/null @@ -1,149 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 17 -toc_title: "Cliente de l\xEDnea de comandos" ---- - -# Cliente de línea de comandos {#command-line-client} - -ClickHouse proporciona un cliente de línea de comandos nativo: `clickhouse-client`. El cliente admite opciones de línea de comandos y archivos de configuración. Para obtener más información, consulte [Configuración](#interfaces_cli_configuration). - -[Instalar](../getting-started/index.md) desde el `clickhouse-client` paquete y ejecútelo con el comando `clickhouse-client`. - -``` bash -$ clickhouse-client -ClickHouse client version 19.17.1.1579 (official build). -Connecting to localhost:9000 as user default. -Connected to ClickHouse server version 19.17.1 revision 54428. - -:) -``` - -Las diferentes versiones de cliente y servidor son compatibles entre sí, pero es posible que algunas funciones no estén disponibles en clientes anteriores. Se recomienda utilizar la misma versión del cliente que la aplicación de servidor. Cuando intenta usar un cliente de la versión anterior, entonces el servidor, `clickhouse-client` muestra el mensaje: - - ClickHouse client version is older than ClickHouse server. It may lack support for new features. - -## Uso {#cli_usage} - -El cliente se puede utilizar en modo interactivo y no interactivo (por lotes). Para utilizar el modo por lotes, especifique el ‘query’ parámetro, o enviar datos a ‘stdin’ (verifica que ‘stdin’ no es un terminal), o ambos. Similar a la interfaz HTTP, cuando se utiliza el ‘query’ parámetro y el envío de datos a ‘stdin’ la solicitud es una concatenación de la ‘query’ parámetro, un avance de línea y los datos en ‘stdin’. Esto es conveniente para grandes consultas INSERT. - -Ejemplo de uso del cliente para insertar datos: - -``` bash -$ echo -ne "1, 'some text', '2016-08-14 00:00:00'\n2, 'some more text', '2016-08-14 00:00:01'" | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV"; - -$ cat <<_EOF | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV"; -3, 'some text', '2016-08-14 00:00:00' -4, 'some more text', '2016-08-14 00:00:01' -_EOF - -$ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV"; -``` - -En el modo por lotes, el formato de datos predeterminado es TabSeparated. Puede establecer el formato en la cláusula FORMAT de la consulta. - -De forma predeterminada, solo puede procesar una única consulta en modo por lotes. Para realizar múltiples consultas desde un “script,” utilizar el `--multiquery` parámetro. Esto funciona para todas las consultas excepto INSERT . Los resultados de la consulta se generan consecutivamente sin separadores adicionales. Del mismo modo, para procesar un gran número de consultas, puede ejecutar ‘clickhouse-client’ para cada consulta. Tenga en cuenta que puede tomar decenas de milisegundos para iniciar el ‘clickhouse-client’ programa. - -En el modo interactivo, obtiene una línea de comandos donde puede ingresar consultas. - -Si ‘multiline’ no se especifica (el valor predeterminado): Para ejecutar la consulta, pulse Intro. El punto y coma no es necesario al final de la consulta. Para introducir una consulta de varias líneas, introduzca una barra invertida `\` antes de la alimentación de línea. Después de presionar Enter, se le pedirá que ingrese la siguiente línea de la consulta. - -Si se especifica multilínea: Para ejecutar una consulta, finalícela con un punto y coma y presione Intro. Si se omitió el punto y coma al final de la línea ingresada, se le pedirá que ingrese la siguiente línea de la consulta. - -Solo se ejecuta una sola consulta, por lo que se ignora todo después del punto y coma. - -Puede especificar `\G` en lugar o después del punto y coma. Esto indica el formato vertical. En este formato, cada valor se imprime en una línea separada, lo cual es conveniente para tablas anchas. Esta característica inusual se agregó por compatibilidad con la CLI de MySQL. - -La línea de comandos se basa en ‘replxx’ (similar a ‘readline’). En otras palabras, utiliza los atajos de teclado familiares y mantiene un historial. La historia está escrita para `~/.clickhouse-client-history`. - -De forma predeterminada, el formato utilizado es PrettyCompact. Puede cambiar el formato en la cláusula FORMAT de la consulta o especificando `\G` al final de la consulta, utilizando el `--format` o `--vertical` en la línea de comandos, o utilizando el archivo de configuración del cliente. - -Para salir del cliente, presione Ctrl+D o introduzca una de las siguientes opciones en lugar de una consulta: “exit”, “quit”, “logout”, “exit;”, “quit;”, “logout;”, “q”, “Q”, “:q” - -Al procesar una consulta, el cliente muestra: - -1. Progreso, que se actualiza no más de 10 veces por segundo (de forma predeterminada). Para consultas rápidas, es posible que el progreso no tenga tiempo para mostrarse. -2. La consulta con formato después del análisis, para la depuración. -3. El resultado en el formato especificado. -4. El número de líneas en el resultado, el tiempo transcurrido y la velocidad promedio de procesamiento de consultas. - -Puede cancelar una consulta larga presionando Ctrl + C. Sin embargo, aún tendrá que esperar un poco para que el servidor aborte la solicitud. No es posible cancelar una consulta en determinadas etapas. Si no espera y presiona Ctrl + C por segunda vez, el cliente saldrá. - -El cliente de línea de comandos permite pasar datos externos (tablas temporales externas) para consultar. Para obtener más información, consulte la sección “External data for query processing”. - -### Consultas con parámetros {#cli-queries-with-parameters} - -Puede crear una consulta con parámetros y pasarles valores desde la aplicación cliente. Esto permite evitar formatear consultas con valores dinámicos específicos en el lado del cliente. Por ejemplo: - -``` bash -$ clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" -``` - -#### Sintaxis de consulta {#cli-queries-with-parameters-syntax} - -Formatee una consulta como de costumbre, luego coloque los valores que desea pasar de los parámetros de la aplicación a la consulta entre llaves en el siguiente formato: - -``` sql -{:} -``` - -- `name` — Placeholder identifier. In the console client it should be used in app parameters as `--param_ = value`. -- `data type` — [Tipo de datos](../sql-reference/data-types/index.md) del valor del parámetro de la aplicación. Por ejemplo, una estructura de datos como `(integer, ('string', integer))` puede tener el `Tuple(UInt8, Tuple(String, UInt8))` tipo de datos (también puede usar otro [entero](../sql-reference/data-types/int-uint.md) tipo). - -#### Ejemplo {#example} - -``` bash -$ clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}" -``` - -## Configuración {#interfaces_cli_configuration} - -Puede pasar parámetros a `clickhouse-client` (todos los parámetros tienen un valor predeterminado) usando: - -- Desde la línea de comandos - - Las opciones de la línea de comandos anulan los valores y valores predeterminados de los archivos de configuración. - -- Archivos de configuración. - - Los valores de los archivos de configuración anulan los valores predeterminados. - -### Opciones de línea de comandos {#command-line-options} - -- `--host, -h` -– The server name, ‘localhost’ predeterminada. Puede utilizar el nombre o la dirección IPv4 o IPv6. -- `--port` – The port to connect to. Default value: 9000. Note that the HTTP interface and the native interface use different ports. -- `--user, -u` – The username. Default value: default. -- `--password` – The password. Default value: empty string. -- `--query, -q` – The query to process when using non-interactive mode. -- `--database, -d` – Select the current default database. Default value: the current database from the server settings (‘default’ predeterminada). -- `--multiline, -m` – If specified, allow multiline queries (do not send the query on Enter). -- `--multiquery, -n` – If specified, allow processing multiple queries separated by semicolons. -- `--format, -f` – Use the specified default format to output the result. -- `--vertical, -E` – If specified, use the Vertical format by default to output the result. This is the same as ‘–format=Vertical’. En este formato, cada valor se imprime en una línea separada, lo que es útil cuando se muestran tablas anchas. -- `--time, -t` – If specified, print the query execution time to ‘stderr’ en modo no interactivo. -- `--stacktrace` – If specified, also print the stack trace if an exception occurs. -- `--config-file` – The name of the configuration file. -- `--secure` – If specified, will connect to server over secure connection. -- `--param_` — Value for a [consulta con parámetros](#cli-queries-with-parameters). - -### Archivos de configuración {#configuration_files} - -`clickhouse-client` utiliza el primer archivo existente de los siguientes: - -- Definido en el `--config-file` parámetro. -- `./clickhouse-client.xml` -- `~/.clickhouse-client/config.xml` -- `/etc/clickhouse-client/config.xml` - -Ejemplo de un archivo de configuración: - -``` xml - - username - password - False - -``` - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/cli/) diff --git a/docs/es/interfaces/cpp.md b/docs/es/interfaces/cpp.md deleted file mode 100644 index bc5dc3dbc24..00000000000 --- a/docs/es/interfaces/cpp.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 24 -toc_title: Biblioteca de clientes de C++ ---- - -# Biblioteca de clientes de C++ {#c-client-library} - -Ver README en [Bienvenidos](https://github.com/ClickHouse/clickhouse-cpp) repositorio. - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/cpp/) diff --git a/docs/es/interfaces/formats.md b/docs/es/interfaces/formats.md deleted file mode 100644 index 03c1873d306..00000000000 --- a/docs/es/interfaces/formats.md +++ /dev/null @@ -1,1212 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 21 -toc_title: Formatos de entrada y salida ---- - -# Formatos para datos de entrada y salida {#formats} - -ClickHouse puede aceptar y devolver datos en varios formatos. Se puede utilizar un formato admitido para la entrada para analizar los datos proporcionados a `INSERT`s, para llevar a cabo `SELECT`s de una tabla respaldada por archivos como File, URL o HDFS, o para leer un diccionario externo. Se puede utilizar un formato compatible con la salida para organizar el -resultados de un `SELECT`, y realizar `INSERT`s en una tabla respaldada por archivos. - -Los formatos soportados son: - -| Formato | Entrada | Salida | -|-----------------------------------------------------------------|---------|--------| -| [TabSeparated](#tabseparated) | ✔ | ✔ | -| [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | -| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | -| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Plantilla](#format-template) | ✔ | ✔ | -| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | -| [CSV](#csv) | ✔ | ✔ | -| [CSVWithNames](#csvwithnames) | ✔ | ✔ | -| [CustomSeparated](#format-customseparated) | ✔ | ✔ | -| [Valor](#data-format-values) | ✔ | ✔ | -| [Vertical](#vertical) | ✗ | ✔ | -| [VerticalRaw](#verticalraw) | ✗ | ✔ | -| [JSON](#json) | ✗ | ✔ | -| [JSONCompact](#jsoncompact) | ✗ | ✔ | -| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | -| [TSKV](#tskv) | ✔ | ✔ | -| [Bastante](#pretty) | ✗ | ✔ | -| [PrettyCompact](#prettycompact) | ✗ | ✔ | -| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | -| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | -| [Bienvenido a WordPress.](#prettyspace) | ✗ | ✔ | -| [Protobuf](#protobuf) | ✔ | ✔ | -| [Avro](#data-format-avro) | ✔ | ✔ | -| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | -| [Parquet](#data-format-parquet) | ✔ | ✔ | -| [ORC](#data-format-orc) | ✔ | ✗ | -| [RowBinary](#rowbinary) | ✔ | ✔ | -| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | -| [Nativo](#native) | ✔ | ✔ | -| [Nulo](#null) | ✗ | ✔ | -| [XML](#xml) | ✗ | ✔ | -| [CapnProto](#capnproto) | ✔ | ✗ | - -Puede controlar algunos parámetros de procesamiento de formato con la configuración de ClickHouse. Para obtener más información, lea el [Configuración](../operations/settings/settings.md) apartado. - -## TabSeparated {#tabseparated} - -En el formato TabSeparated, los datos se escriben por fila. Cada fila contiene valores separados por pestañas. Cada valor es seguido por una ficha, excepto el último valor de la fila, que es seguido por un avance de línea. Estrictamente las fuentes de línea Unix se asumen en todas partes. La última fila también debe contener un avance de línea al final. Los valores se escriben en formato de texto, sin incluir comillas y con caracteres especiales escapados. - -Este formato también está disponible bajo el nombre `TSV`. - -El `TabSeparated` es conveniente para procesar datos utilizando programas y scripts personalizados. Se usa de forma predeterminada en la interfaz HTTP y en el modo por lotes del cliente de línea de comandos. Este formato también permite transferir datos entre diferentes DBMS. Por ejemplo, puede obtener un volcado de MySQL y subirlo a ClickHouse, o viceversa. - -El `TabSeparated` el formato admite la salida de valores totales (cuando se usa WITH TOTALS) y valores extremos (cuando ‘extremes’ se establece en 1). En estos casos, los valores totales y los extremos se emiten después de los datos principales. El resultado principal, los valores totales y los extremos están separados entre sí por una línea vacía. Ejemplo: - -``` sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` -``` - -``` text -2014-03-17 1406958 -2014-03-18 1383658 -2014-03-19 1405797 -2014-03-20 1353623 -2014-03-21 1245779 -2014-03-22 1031592 -2014-03-23 1046491 - -1970-01-01 8873898 - -2014-03-17 1031592 -2014-03-23 1406958 -``` - -### Formato de datos {#data-formatting} - -Los números enteros se escriben en forma decimal. Los números pueden contener un extra “+” carácter al principio (ignorado al analizar y no grabado al formatear). Los números no negativos no pueden contener el signo negativo. Al leer, se permite analizar una cadena vacía como cero, o (para tipos con signo) una cadena que consiste en solo un signo menos como cero. Los números que no encajan en el tipo de datos correspondiente se pueden analizar como un número diferente, sin un mensaje de error. - -Los números de punto flotante se escriben en forma decimal. El punto se usa como separador decimal. Las entradas exponenciales son compatibles, al igual que ‘inf’, ‘+inf’, ‘-inf’, y ‘nan’. Una entrada de números de coma flotante puede comenzar o terminar con un punto decimal. -Durante el formateo, la precisión puede perderse en los números de coma flotante. -Durante el análisis, no es estrictamente necesario leer el número representable de la máquina más cercano. - -Las fechas se escriben en formato AAAA-MM-DD y se analizan en el mismo formato, pero con los caracteres como separadores. -Las fechas con horas se escriben en el formato `YYYY-MM-DD hh:mm:ss` y analizado en el mismo formato, pero con cualquier carácter como separadores. -Todo esto ocurre en la zona horaria del sistema en el momento en que se inicia el cliente o servidor (dependiendo de cuál de ellos formatea los datos). Para fechas con horarios, no se especifica el horario de verano. Por lo tanto, si un volcado tiene tiempos durante el horario de verano, el volcado no coincide inequívocamente con los datos, y el análisis seleccionará una de las dos veces. -Durante una operación de lectura, las fechas incorrectas y las fechas con horas se pueden analizar con desbordamiento natural o como fechas y horas nulas, sin un mensaje de error. - -Como excepción, el análisis de fechas con horas también se admite en el formato de marca de tiempo Unix, si consta de exactamente 10 dígitos decimales. El resultado no depende de la zona horaria. Los formatos AAAA-MM-DD hh:mm:ss y NNNNNNNNNN se diferencian automáticamente. - -Las cadenas se generan con caracteres especiales de escape de barra invertida. Las siguientes secuencias de escape se utilizan para la salida: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. El análisis también admite las secuencias `\a`, `\v`, y `\xHH` (secuencias de escape hexagonales) y cualquier `\c` secuencias, donde `c` es cualquier carácter (estas secuencias se convierten en `c`). Por lo tanto, la lectura de datos admite formatos donde un avance de línea se puede escribir como `\n` o `\` o como un avance de línea. Por ejemplo, la cadena `Hello world` con un avance de línea entre las palabras en lugar de espacio se puede analizar en cualquiera de las siguientes variaciones: - -``` text -Hello\nworld - -Hello\ -world -``` - -La segunda variante es compatible porque MySQL la usa al escribir volcados separados por tabuladores. - -El conjunto mínimo de caracteres que debe escapar al pasar datos en formato TabSeparated: tabulación, salto de línea (LF) y barra invertida. - -Solo se escapa un pequeño conjunto de símbolos. Puede tropezar fácilmente con un valor de cadena que su terminal arruinará en la salida. - -Las matrices se escriben como una lista de valores separados por comas entre corchetes. Los elementos numéricos de la matriz tienen el formato normal. `Date` y `DateTime` están escritos entre comillas simples. Las cadenas se escriben entre comillas simples con las mismas reglas de escape que las anteriores. - -[NULL](../sql-reference/syntax.md) se formatea como `\N`. - -Cada elemento de [Anidar](../sql-reference/data-types/nested-data-structures/nested.md) estructuras se representa como una matriz. - -Por ejemplo: - -``` sql -CREATE TABLE nestedt -( - `id` UInt8, - `aux` Nested( - a UInt8, - b String - ) -) -ENGINE = TinyLog -``` - -``` sql -INSERT INTO nestedt Values ( 1, [1], ['a']) -``` - -``` sql -SELECT * FROM nestedt FORMAT TSV -``` - -``` text -1 [1] ['a'] -``` - -## TabSeparatedRaw {#tabseparatedraw} - -Difiere de `TabSeparated` formato en que las filas se escriben sin escapar. -Este formato solo es apropiado para generar un resultado de consulta, pero no para analizar (recuperar datos para insertar en una tabla). - -Este formato también está disponible bajo el nombre `TSVRaw`. - -## TabSeparatedWithNames {#tabseparatedwithnames} - -Difiere de la `TabSeparated` formato en que los nombres de columna se escriben en la primera fila. -Durante el análisis, la primera fila se ignora por completo. No puede usar nombres de columna para determinar su posición o para comprobar su corrección. -(Se puede agregar soporte para analizar la fila de encabezado en el futuro.) - -Este formato también está disponible bajo el nombre `TSVWithNames`. - -## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} - -Difiere de la `TabSeparated` formato en que los nombres de columna se escriben en la primera fila, mientras que los tipos de columna están en la segunda fila. -Durante el análisis, la primera y la segunda filas se ignoran por completo. - -Este formato también está disponible bajo el nombre `TSVWithNamesAndTypes`. - -## Plantilla {#format-template} - -Este formato permite especificar una cadena de formato personalizado con marcadores de posición para los valores con una regla de escape especificada. - -Utiliza la configuración `format_template_resultset`, `format_template_row`, `format_template_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` cuando se utiliza `JSON` escapar, ver más) - -Configuración `format_template_row` especifica la ruta de acceso al archivo, que contiene una cadena de formato para las filas con la siguiente sintaxis: - -`delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`, - -donde `delimiter_i` es un delimitador entre valores (`$` símbolo se puede escapar como `$$`), -`column_i` es un nombre o índice de una columna cuyos valores se deben seleccionar o insertar (si está vacío, se omitirá la columna), -`serializeAs_i` es una regla de escape para los valores de columna. Se admiten las siguientes reglas de escape: - -- `CSV`, `JSON`, `XML` (similar a los formatos de los mismos nombres) -- `Escaped` (similar a `TSV`) -- `Quoted` (similar a `Values`) -- `Raw` (sin escapar, de manera similar a `TSVRaw`) -- `None` (sin regla de escape, ver más) - -Si se omite una regla de escape, entonces `None` se utilizará. `XML` y `Raw` son adecuados sólo para la salida. - -Entonces, para la siguiente cadena de formato: - - `Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};` - -los valores de `SearchPhrase`, `c` y `price` columnas, que se escapan como `Quoted`, `Escaped` y `JSON` se imprimirá (para seleccionar) o se esperará (para insertar) entre `Search phrase:`, `, count:`, `, ad price: $` y `;` delimitadores respectivamente. Por ejemplo: - -`Search phrase: 'bathroom interior design', count: 2166, ad price: $3;` - -El `format_template_rows_between_delimiter` setting especifica el delimitador entre filas, que se imprime (o se espera) después de cada fila, excepto la última (`\n` predeterminada) - -Configuración `format_template_resultset` especifica la ruta al archivo, que contiene una cadena de formato para el conjunto de resultados. La cadena de formato para el conjunto de resultados tiene la misma sintaxis que una cadena de formato para la fila y permite especificar un prefijo, un sufijo y una forma de imprimir información adicional. Contiene los siguientes marcadores de posición en lugar de nombres de columna: - -- `data` son las filas con datos en `format_template_row` formato, separados por `format_template_rows_between_delimiter`. Este marcador de posición debe ser el primer marcador de posición en la cadena de formato. -- `totals` es la fila con valores totales en `format_template_row` formato (cuando se usa WITH TOTALS) -- `min` es la fila con valores mínimos en `format_template_row` formato (cuando los extremos se establecen en 1) -- `max` es la fila con valores máximos en `format_template_row` formato (cuando los extremos se establecen en 1) -- `rows` es el número total de filas de salida -- `rows_before_limit` es el número mínimo de filas que habría habido sin LIMIT. Salida solo si la consulta contiene LIMIT. Si la consulta contiene GROUP BY, rows_before_limit_at_least es el número exacto de filas que habría habido sin un LIMIT . -- `time` es el tiempo de ejecución de la solicitud en segundos -- `rows_read` es el número de filas que se ha leído -- `bytes_read` es el número de bytes (sin comprimir) que se ha leído - -Marcador `data`, `totals`, `min` y `max` no debe tener una regla de escape especificada (o `None` debe especificarse explícitamente). Los marcadores de posición restantes pueden tener cualquier regla de escape especificada. -Si el `format_template_resultset` valor es una cadena vacía, `${data}` se utiliza como valor predeterminado. -Para el formato de consultas de inserción permite omitir algunas columnas o algunos campos si prefijo o sufijo (ver ejemplo). - -Seleccionar ejemplo: - -``` sql -SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5 FORMAT Template SETTINGS -format_template_resultset = '/some/path/resultset.format', format_template_row = '/some/path/row.format', format_template_rows_between_delimiter = '\n ' -``` - -`/some/path/resultset.format`: - -``` text - - Search phrases - - - - ${data} -
Search phrases
Search phrase Count
- - ${max} -
Max
- Processed ${rows_read:XML} rows in ${time:XML} sec - - -``` - -`/some/path/row.format`: - -``` text - ${0:XML} ${1:XML} -``` - -Resultado: - -``` html - - Search phrases - - - - - - - - -
Search phrases
Search phrase Count
8267016
bathroom interior design 2166
yandex 1655
spring 2014 fashion 1549
freeform photos 1480
- - -
Max
8873898
- Processed 3095973 rows in 0.1569913 sec - - -``` - -Insertar ejemplo: - -``` text -Some header -Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1 -Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1 -Total rows: 2 -``` - -``` sql -INSERT INTO UserActivity FORMAT Template SETTINGS -format_template_resultset = '/some/path/resultset.format', format_template_row = '/some/path/row.format' -``` - -`/some/path/resultset.format`: - -``` text -Some header\n${data}\nTotal rows: ${:CSV}\n -``` - -`/some/path/row.format`: - -``` text -Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV} -``` - -`PageViews`, `UserID`, `Duration` y `Sign` dentro de los marcadores de posición son nombres de columnas en la tabla. Valores después `Useless field` en filas y después `\nTotal rows:` en el sufijo será ignorado. -Todos los delimitadores de los datos de entrada deben ser estrictamente iguales a los delimitadores de las cadenas de formato especificadas. - -## TemplateIgnoreSpaces {#templateignorespaces} - -Este formato es adecuado sólo para la entrada. -Similar a `Template`, pero omite caracteres de espacio en blanco entre delimitadores y valores en la secuencia de entrada. Sin embargo, si las cadenas de formato contienen caracteres de espacio en blanco, se esperarán estos caracteres en la secuencia de entrada. También permite especificar marcadores de posición vacíos (`${}` o `${:None}`) para dividir algún delimitador en partes separadas para ignorar los espacios entre ellos. Dichos marcadores de posición se usan solo para omitir caracteres de espacio en blanco. -Es posible leer `JSON` usando este formato, si los valores de las columnas tienen el mismo orden en todas las filas. Por ejemplo, la siguiente solicitud se puede utilizar para insertar datos del ejemplo de salida de formato [JSON](#json): - -``` sql -INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS -format_template_resultset = '/some/path/resultset.format', format_template_row = '/some/path/row.format', format_template_rows_between_delimiter = ',' -``` - -`/some/path/resultset.format`: - -``` text -{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}} -``` - -`/some/path/row.format`: - -``` text -{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}} -``` - -## TSKV {#tskv} - -Similar a TabSeparated , pero genera un valor en formato name=value . Los nombres se escapan de la misma manera que en el formato TabSeparated, y el símbolo = también se escapa. - -``` text -SearchPhrase= count()=8267016 -SearchPhrase=bathroom interior design count()=2166 -SearchPhrase=yandex count()=1655 -SearchPhrase=2014 spring fashion count()=1549 -SearchPhrase=freeform photos count()=1480 -SearchPhrase=angelina jolie count()=1245 -SearchPhrase=omsk count()=1112 -SearchPhrase=photos of dog breeds count()=1091 -SearchPhrase=curtain designs count()=1064 -SearchPhrase=baku count()=1000 -``` - -[NULL](../sql-reference/syntax.md) se formatea como `\N`. - -``` sql -SELECT * FROM t_null FORMAT TSKV -``` - -``` text -x=1 y=\N -``` - -Cuando hay una gran cantidad de columnas pequeñas, este formato no es efectivo y generalmente no hay razón para usarlo. Sin embargo, no es peor que JSONEachRow en términos de eficiencia. - -Both data output and parsing are supported in this format. For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted – they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. - -El análisis permite la presencia del campo adicional `tskv` sin el signo igual o un valor. Este campo se ignora. - -## CSV {#csv} - -Formato de valores separados por comas ([RFC](https://tools.ietf.org/html/rfc4180)). - -Al formatear, las filas están encerradas en comillas dobles. Una comilla doble dentro de una cadena se genera como dos comillas dobles en una fila. No hay otras reglas para escapar de los personajes. Fecha y fecha-hora están encerrados en comillas dobles. Los números se emiten sin comillas. Los valores están separados por un carácter delimitador, que es `,` predeterminada. El carácter delimitador se define en la configuración [Formato_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Las filas se separan usando el avance de línea Unix (LF). Las matrices se serializan en CSV de la siguiente manera: primero, la matriz se serializa en una cadena como en el formato TabSeparated, y luego la cadena resultante se envía a CSV en comillas dobles. Las tuplas en formato CSV se serializan como columnas separadas (es decir, se pierde su anidamiento en la tupla). - -``` bash -$ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv -``` - -\*De forma predeterminada, el delimitador es `,`. Ver el [Formato_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) para obtener más información. - -Al analizar, todos los valores se pueden analizar con o sin comillas. Ambas comillas dobles y simples son compatibles. Las filas también se pueden organizar sin comillas. En este caso, se analizan hasta el carácter delimitador o el avance de línea (CR o LF). En violación del RFC, al analizar filas sin comillas, se ignoran los espacios y pestañas iniciales y finales. Para el avance de línea, se admiten los tipos Unix (LF), Windows (CR LF) y Mac OS Classic (CR LF). - -Los valores de entrada vacíos sin comillas se sustituyen por valores predeterminados para las columnas respectivas, si -[Entrada_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) -está habilitado. - -`NULL` se formatea como `\N` o `NULL` o una cadena vacía sin comillas (consulte la configuración [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) y [Entrada_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). - -El formato CSV admite la salida de totales y extremos de la misma manera que `TabSeparated`. - -## CSVWithNames {#csvwithnames} - -También imprime la fila del encabezado, similar a `TabSeparatedWithNames`. - -## CustomSeparated {#format-customseparated} - -Similar a [Plantilla](#format-template), pero imprime o lee todas las columnas y usa la regla de escape de la configuración `format_custom_escaping_rule` y delimitadores desde la configuración `format_custom_field_delimiter`, `format_custom_row_before_delimiter`, `format_custom_row_after_delimiter`, `format_custom_row_between_delimiter`, `format_custom_result_before_delimiter` y `format_custom_result_after_delimiter`, no de cadenas de formato. -También hay `CustomSeparatedIgnoreSpaces` formato, que es similar a `TemplateIgnoreSpaces`. - -## JSON {#json} - -Salida de datos en formato JSON. Además de las tablas de datos, también genera nombres y tipos de columnas, junto con información adicional: el número total de filas de salida y el número de filas que podrían haberse generado si no hubiera un LIMIT . Ejemplo: - -``` sql -SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTALS ORDER BY c DESC LIMIT 5 FORMAT JSON -``` - -``` json -{ - "meta": - [ - { - "name": "SearchPhrase", - "type": "String" - }, - { - "name": "c", - "type": "UInt64" - } - ], - - "data": - [ - { - "SearchPhrase": "", - "c": "8267016" - }, - { - "SearchPhrase": "bathroom interior design", - "c": "2166" - }, - { - "SearchPhrase": "yandex", - "c": "1655" - }, - { - "SearchPhrase": "spring 2014 fashion", - "c": "1549" - }, - { - "SearchPhrase": "freeform photos", - "c": "1480" - } - ], - - "totals": - { - "SearchPhrase": "", - "c": "8873898" - }, - - "extremes": - { - "min": - { - "SearchPhrase": "", - "c": "1480" - }, - "max": - { - "SearchPhrase": "", - "c": "8267016" - } - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 -} -``` - -El JSON es compatible con JavaScript. Para garantizar esto, algunos caracteres se escapan adicionalmente: la barra inclinada `/` se escapa como `\/`; saltos de línea alternativos `U+2028` y `U+2029`, que rompen algunos navegadores, se escapan como `\uXXXX`. Los caracteres de control ASCII se escapan: retroceso, avance de formulario, avance de línea, retorno de carro y tabulación horizontal se reemplazan con `\b`, `\f`, `\n`, `\r`, `\t` , así como los bytes restantes en el rango 00-1F usando `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double-quotes by default. To remove the quotes, you can set the configuration parameter [output_format_json_quote_64bit_integers](../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) a 0. - -`rows` – The total number of output rows. - -`rows_before_limit_at_least` El número mínimo de filas habría sido sin LIMIT . Salida solo si la consulta contiene LIMIT. -Si la consulta contiene GROUP BY, rows_before_limit_at_least es el número exacto de filas que habría habido sin un LIMIT . - -`totals` – Total values (when using WITH TOTALS). - -`extremes` – Extreme values (when extremes are set to 1). - -Este formato solo es apropiado para generar un resultado de consulta, pero no para analizar (recuperar datos para insertar en una tabla). - -Soporta ClickHouse [NULL](../sql-reference/syntax.md), que se muestra como `null` en la salida JSON. - -Ver también el [JSONEachRow](#jsoneachrow) formato. - -## JSONCompact {#jsoncompact} - -Difiere de JSON solo en que las filas de datos se generan en matrices, no en objetos. - -Ejemplo: - -``` json -{ - "meta": - [ - { - "name": "SearchPhrase", - "type": "String" - }, - { - "name": "c", - "type": "UInt64" - } - ], - - "data": - [ - ["", "8267016"], - ["bathroom interior design", "2166"], - ["yandex", "1655"], - ["fashion trends spring 2014", "1549"], - ["freeform photo", "1480"] - ], - - "totals": ["","8873898"], - - "extremes": - { - "min": ["","1480"], - "max": ["","8267016"] - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 -} -``` - -Este formato solo es apropiado para generar un resultado de consulta, pero no para analizar (recuperar datos para insertar en una tabla). -Ver también el `JSONEachRow` formato. - -## JSONEachRow {#jsoneachrow} - -Al usar este formato, ClickHouse genera filas como objetos JSON separados, delimitados por nuevas líneas, pero los datos en su conjunto no son JSON válidos. - -``` json -{"SearchPhrase":"curtain designs","count()":"1064"} -{"SearchPhrase":"baku","count()":"1000"} -{"SearchPhrase":"","count()":"8267016"} -``` - -Al insertar los datos, debe proporcionar un objeto JSON independiente para cada fila. - -### Insertar datos {#inserting-data} - -``` sql -INSERT INTO UserActivity FORMAT JSONEachRow {"PageViews":5, "UserID":"4324182021466249494", "Duration":146,"Sign":-1} {"UserID":"4324182021466249494","PageViews":6,"Duration":185,"Sign":1} -``` - -ClickHouse permite: - -- Cualquier orden de pares clave-valor en el objeto. -- Omitiendo algunos valores. - -ClickHouse ignora los espacios entre los elementos y las comas después de los objetos. Puede pasar todos los objetos en una línea. No tiene que separarlos con saltos de línea. - -**Procesamiento de valores omitidos** - -ClickHouse sustituye los valores omitidos por los valores predeterminados para el [tipos de datos](../sql-reference/data-types/index.md). - -Si `DEFAULT expr` se especifica, ClickHouse utiliza diferentes reglas de sustitución dependiendo de la [Entrada_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) configuración. - -Considere la siguiente tabla: - -``` sql -CREATE TABLE IF NOT EXISTS example_table -( - x UInt32, - a DEFAULT x * 2 -) ENGINE = Memory; -``` - -- Si `input_format_defaults_for_omitted_fields = 0`, entonces el valor predeterminado para `x` y `a` igual `0` (como el valor predeterminado para el `UInt32` tipo de datos). -- Si `input_format_defaults_for_omitted_fields = 1`, entonces el valor predeterminado para `x` igual `0` pero el valor predeterminado de `a` igual `x * 2`. - -!!! note "Advertencia" - Al insertar datos con `insert_sample_with_metadata = 1`, ClickHouse consume más recursos computacionales, en comparación con la inserción con `insert_sample_with_metadata = 0`. - -### Selección de datos {#selecting-data} - -Considere el `UserActivity` tabla como un ejemplo: - -``` text -┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ -│ 4324182021466249494 │ 5 │ 146 │ -1 │ -│ 4324182021466249494 │ 6 │ 185 │ 1 │ -└─────────────────────┴───────────┴──────────┴──────┘ -``` - -Consulta `SELECT * FROM UserActivity FORMAT JSONEachRow` devoluciones: - -``` text -{"UserID":"4324182021466249494","PageViews":5,"Duration":146,"Sign":-1} -{"UserID":"4324182021466249494","PageViews":6,"Duration":185,"Sign":1} -``` - -A diferencia de la [JSON](#json) formato, no hay sustitución de secuencias UTF-8 no válidas. Los valores se escapan de la misma manera que para `JSON`. - -!!! note "Nota" - Cualquier conjunto de bytes se puede generar en las cadenas. Utilice el `JSONEachRow` si está seguro de que los datos de la tabla se pueden formatear como JSON sin perder ninguna información. - -### Uso de estructuras anidadas {#jsoneachrow-nested} - -Si tienes una mesa con [Anidar](../sql-reference/data-types/nested-data-structures/nested.md) columnas de tipo de datos, puede insertar datos JSON con la misma estructura. Habilite esta función con el [Entrada_format_import_nested_json](../operations/settings/settings.md#settings-input_format_import_nested_json) configuración. - -Por ejemplo, considere la siguiente tabla: - -``` sql -CREATE TABLE json_each_row_nested (n Nested (s String, i Int32) ) ENGINE = Memory -``` - -Como se puede ver en el `Nested` descripción del tipo de datos, ClickHouse trata cada componente de la estructura anidada como una columna separada (`n.s` y `n.i` para nuestra mesa). Puede insertar datos de la siguiente manera: - -``` sql -INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n.s": ["abc", "def"], "n.i": [1, 23]} -``` - -Para insertar datos como un objeto JSON jerárquico, establezca [input_format_import_nested_json=1](../operations/settings/settings.md#settings-input_format_import_nested_json). - -``` json -{ - "n": { - "s": ["abc", "def"], - "i": [1, 23] - } -} -``` - -Sin esta configuración, ClickHouse produce una excepción. - -``` sql -SELECT name, value FROM system.settings WHERE name = 'input_format_import_nested_json' -``` - -``` text -┌─name────────────────────────────┬─value─┐ -│ input_format_import_nested_json │ 0 │ -└─────────────────────────────────┴───────┘ -``` - -``` sql -INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], "i": [1, 23]}} -``` - -``` text -Code: 117. DB::Exception: Unknown field found while parsing JSONEachRow format: n: (at row 1) -``` - -``` sql -SET input_format_import_nested_json=1 -INSERT INTO json_each_row_nested FORMAT JSONEachRow {"n": {"s": ["abc", "def"], "i": [1, 23]}} -SELECT * FROM json_each_row_nested -``` - -``` text -┌─n.s───────────┬─n.i────┐ -│ ['abc','def'] │ [1,23] │ -└───────────────┴────────┘ -``` - -## Nativo {#native} - -El formato más eficiente. Los datos son escritos y leídos por bloques en formato binario. Para cada bloque, el número de filas, número de columnas, nombres y tipos de columnas y partes de columnas de este bloque se registran una tras otra. En otras palabras, este formato es “columnar” – it doesn't convert columns to rows. This is the format used in the native interface for interaction between servers, for using the command-line client, and for C++ clients. - -Puede utilizar este formato para generar rápidamente volcados que sólo pueden ser leídos por el DBMS de ClickHouse. No tiene sentido trabajar con este formato usted mismo. - -## Nulo {#null} - -Nada es salida. Sin embargo, la consulta se procesa y, cuando se utiliza el cliente de línea de comandos, los datos se transmiten al cliente. Esto se usa para pruebas, incluidas las pruebas de rendimiento. -Obviamente, este formato solo es apropiado para la salida, no para el análisis. - -## Bastante {#pretty} - -Salidas de datos como tablas de arte Unicode, también utilizando secuencias de escape ANSI para establecer colores en el terminal. -Se dibuja una cuadrícula completa de la tabla, y cada fila ocupa dos líneas en la terminal. -Cada bloque de resultados se muestra como una tabla separada. Esto es necesario para que los bloques se puedan generar sin resultados de almacenamiento en búfer (el almacenamiento en búfer sería necesario para calcular previamente el ancho visible de todos los valores). - -[NULL](../sql-reference/syntax.md) se emite como `ᴺᵁᴸᴸ`. - -Ejemplo (mostrado para el [PrettyCompact](#prettycompact) formato): - -``` sql -SELECT * FROM t_null -``` - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -└───┴──────┘ -``` - -Las filas no se escapan en formatos Pretty \*. Se muestra un ejemplo para el [PrettyCompact](#prettycompact) formato: - -``` sql -SELECT 'String with \'quotes\' and \t character' AS Escaping_test -``` - -``` text -┌─Escaping_test────────────────────────┐ -│ String with 'quotes' and character │ -└──────────────────────────────────────┘ -``` - -Para evitar volcar demasiados datos al terminal, solo se imprimen las primeras 10.000 filas. Si el número de filas es mayor o igual que 10.000, el mensaje “Showed first 10 000” se imprime. -Este formato solo es apropiado para generar un resultado de consulta, pero no para analizar (recuperar datos para insertar en una tabla). - -El formato Pretty admite la salida de valores totales (cuando se usa WITH TOTALS) y extremos (cuando ‘extremes’ se establece en 1). En estos casos, los valores totales y los valores extremos se generan después de los datos principales, en tablas separadas. Ejemplo (mostrado para el [PrettyCompact](#prettycompact) formato): - -``` sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact -``` - -``` text -┌──EventDate─┬───────c─┐ -│ 2014-03-17 │ 1406958 │ -│ 2014-03-18 │ 1383658 │ -│ 2014-03-19 │ 1405797 │ -│ 2014-03-20 │ 1353623 │ -│ 2014-03-21 │ 1245779 │ -│ 2014-03-22 │ 1031592 │ -│ 2014-03-23 │ 1046491 │ -└────────────┴─────────┘ - -Totals: -┌──EventDate─┬───────c─┐ -│ 1970-01-01 │ 8873898 │ -└────────────┴─────────┘ - -Extremes: -┌──EventDate─┬───────c─┐ -│ 2014-03-17 │ 1031592 │ -│ 2014-03-23 │ 1406958 │ -└────────────┴─────────┘ -``` - -## PrettyCompact {#prettycompact} - -Difiere de [Bastante](#pretty) en que la cuadrícula se dibuja entre filas y el resultado es más compacto. -Este formato se usa de forma predeterminada en el cliente de línea de comandos en modo interactivo. - -## PrettyCompactMonoBlock {#prettycompactmonoblock} - -Difiere de [PrettyCompact](#prettycompact) en que hasta 10,000 filas se almacenan en búfer, luego se salen como una sola tabla, no por bloques. - -## PrettyNoEscapes {#prettynoescapes} - -Difiere de Pretty en que las secuencias de escape ANSI no se usan. Esto es necesario para mostrar este formato en un navegador, así como para usar el ‘watch’ utilidad de línea de comandos. - -Ejemplo: - -``` bash -$ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FORMAT PrettyCompactNoEscapes'" -``` - -Puede usar la interfaz HTTP para mostrar en el navegador. - -### PrettyCompactNoEscapes {#prettycompactnoescapes} - -Lo mismo que el ajuste anterior. - -### PrettySpaceNoEscapes {#prettyspacenoescapes} - -Lo mismo que el ajuste anterior. - -## Bienvenido a WordPress {#prettyspace} - -Difiere de [PrettyCompact](#prettycompact) en ese espacio en blanco (caracteres de espacio) se usa en lugar de la cuadrícula. - -## RowBinary {#rowbinary} - -Formatea y analiza datos por fila en formato binario. Las filas y los valores se enumeran consecutivamente, sin separadores. -Este formato es menos eficiente que el formato nativo, ya que está basado en filas. - -Los integradores usan una representación little-endian de longitud fija. Por ejemplo, UInt64 usa 8 bytes. -DateTime se representa como UInt32 que contiene la marca de tiempo Unix como el valor. -Date se representa como un objeto UInt16 que contiene el número de días desde 1970-01-01 como el valor. -La cadena se representa como una longitud varint (sin signo [LEB128](https://en.wikipedia.org/wiki/LEB128)), seguido de los bytes de la cadena. -FixedString se representa simplemente como una secuencia de bytes. - -La matriz se representa como una longitud varint (sin signo [LEB128](https://en.wikipedia.org/wiki/LEB128)), seguido de elementos sucesivos de la matriz. - -Para [NULL](../sql-reference/syntax.md#null-literal) soporte, se añade un byte adicional que contiene 1 o 0 antes de cada [NULL](../sql-reference/data-types/nullable.md) valor. Si 1, entonces el valor es `NULL` y este byte se interpreta como un valor separado. Si es 0, el valor después del byte no es `NULL`. - -## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes} - -Similar a [RowBinary](#rowbinary), pero con encabezado añadido: - -- [LEB128](https://en.wikipedia.org/wiki/LEB128)-número codificado de columnas (N) -- N `String`s especificando nombres de columna -- N `String`s especificando tipos de columna - -## Valor {#data-format-values} - -Imprime cada fila entre paréntesis. Las filas están separadas por comas. No hay coma después de la última fila. Los valores dentro de los corchetes también están separados por comas. Los números se emiten en formato decimal sin comillas. Las matrices se emiten entre corchetes. Las cadenas, fechas y fechas con horas se generan entre comillas. Las reglas de escape y el análisis son similares a las [TabSeparated](#tabseparated) formato. Durante el formateo, los espacios adicionales no se insertan, pero durante el análisis, se permiten y omiten (excepto los espacios dentro de los valores de la matriz, que no están permitidos). [NULL](../sql-reference/syntax.md) se representa como `NULL`. - -The minimum set of characters that you need to escape when passing data in Values ​​format: single quotes and backslashes. - -Este es el formato que se utiliza en `INSERT INTO t VALUES ...`, pero también puede usarlo para formatear los resultados de la consulta. - -Ver también: [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) y [input_format_values_deduce_templates_of_expressions](../operations/settings/settings.md#settings-input_format_values_deduce_templates_of_expressions) configuración. - -## Vertical {#vertical} - -Imprime cada valor en una línea independiente con el nombre de columna especificado. Este formato es conveniente para imprimir solo una o varias filas si cada fila consta de un gran número de columnas. - -[NULL](../sql-reference/syntax.md) se emite como `ᴺᵁᴸᴸ`. - -Ejemplo: - -``` sql -SELECT * FROM t_null FORMAT Vertical -``` - -``` text -Row 1: -────── -x: 1 -y: ᴺᵁᴸᴸ -``` - -Las filas no se escapan en formato vertical: - -``` sql -SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical -``` - -``` text -Row 1: -────── -test: string with 'quotes' and with some special - characters -``` - -Este formato solo es apropiado para generar un resultado de consulta, pero no para analizar (recuperar datos para insertar en una tabla). - -## VerticalRaw {#verticalraw} - -Similar a [Vertical](#vertical), pero con escapar deshabilitado. Este formato solo es adecuado para generar resultados de consultas, no para analizar (recibir datos e insertarlos en la tabla). - -## XML {#xml} - -El formato XML es adecuado solo para la salida, no para el análisis. Ejemplo: - -``` xml - - - - - - SearchPhrase - String - - - count() - UInt64 - - - - - - - 8267016 - - - bathroom interior design - 2166 - - - yandex - 1655 - - - 2014 spring fashion - 1549 - - - freeform photos - 1480 - - - angelina jolie - 1245 - - - omsk - 1112 - - - photos of dog breeds - 1091 - - - curtain designs - 1064 - - - baku - 1000 - - - 10 - 141137 - -``` - -Si el nombre de la columna no tiene un formato aceptable, simplemente ‘field’ se utiliza como el nombre del elemento. En general, la estructura XML sigue la estructura JSON. -Just as for JSON, invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. - -En los valores de cadena, los caracteres `<` y `&` se escaparon como `<` y `&`. - -Las matrices se emiten como `HelloWorld...`y tuplas como `HelloWorld...`. - -## CapnProto {#capnproto} - -Cap'n Proto es un formato de mensaje binario similar a Protocol Buffers y Thrift, pero no como JSON o MessagePack. - -Los mensajes de Cap'n Proto están estrictamente escritos y no autodescribidos, lo que significa que necesitan una descripción de esquema externo. El esquema se aplica sobre la marcha y se almacena en caché para cada consulta. - -``` bash -$ cat capnproto_messages.bin | clickhouse-client --query "INSERT INTO test.hits FORMAT CapnProto SETTINGS format_schema='schema:Message'" -``` - -Donde `schema.capnp` se ve así: - -``` capnp -struct Message { - SearchPhrase @0 :Text; - c @1 :Uint64; -} -``` - -La deserialización es efectiva y generalmente no aumenta la carga del sistema. - -Ver también [Esquema de formato](#formatschema). - -## Protobuf {#protobuf} - -Protobuf - es un [Búferes de protocolo](https://developers.google.com/protocol-buffers/) formato. - -Este formato requiere un esquema de formato externo. El esquema se almacena en caché entre las consultas. -ClickHouse soporta ambos `proto2` y `proto3` sintaxis. Se admiten campos repetidos / opcionales / requeridos. - -Ejemplos de uso: - -``` sql -SELECT * FROM test.table FORMAT Protobuf SETTINGS format_schema = 'schemafile:MessageType' -``` - -``` bash -cat protobuf_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT Protobuf SETTINGS format_schema='schemafile:MessageType'" -``` - -donde el archivo `schemafile.proto` se ve así: - -``` capnp -syntax = "proto3"; - -message MessageType { - string name = 1; - string surname = 2; - uint32 birthDate = 3; - repeated string phoneNumbers = 4; -}; -``` - -Para encontrar la correspondencia entre las columnas de la tabla y los campos del tipo de mensaje de Protocol Buffers, ClickHouse compara sus nombres. -Esta comparación no distingue entre mayúsculas y minúsculas y los caracteres `_` (subrayado) y `.` (punto) se consideran iguales. -Si los tipos de una columna y un campo del mensaje de Protocol Buffers son diferentes, se aplica la conversión necesaria. - -Los mensajes anidados son compatibles. Por ejemplo, para el campo `z` en el siguiente tipo de mensaje - -``` capnp -message MessageType { - message XType { - message YType { - int32 z; - }; - repeated YType y; - }; - XType x; -}; -``` - -ClickHouse intenta encontrar una columna llamada `x.y.z` (o `x_y_z` o `X.y_Z` y así sucesivamente). -Los mensajes anidados son adecuados para [estructuras de datos anidados](../sql-reference/data-types/nested-data-structures/nested.md). - -Valores predeterminados definidos en un esquema protobuf como este - -``` capnp -syntax = "proto2"; - -message MessageType { - optional int32 result_per_page = 3 [default = 10]; -} -``` - -no se aplican; el [valores predeterminados de la tabla](../sql-reference/statements/create.md#create-default-values) se utilizan en lugar de ellos. - -ClickHouse entra y emite mensajes protobuf en el `length-delimited` formato. -Significa que antes de cada mensaje debe escribirse su longitud como un [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints). -Ver también [cómo leer / escribir mensajes protobuf delimitados por longitud en idiomas populares](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages). - -## Avro {#data-format-avro} - -[Más información](http://avro.apache.org/) es un marco de serialización de datos orientado a filas desarrollado dentro del proyecto Hadoop de Apache. - -El formato ClickHouse Avro admite lectura y escritura [Archivos de datos Avro](http://avro.apache.org/docs/current/spec.html#Object+Container+Files). - -### Coincidencia de tipos de datos {#data_types-matching} - -La siguiente tabla muestra los tipos de datos admitidos y cómo coinciden con ClickHouse [tipos de datos](../sql-reference/data-types/index.md) en `INSERT` y `SELECT` consulta. - -| Tipo de datos Avro `INSERT` | Tipo de datos ClickHouse | Tipo de datos Avro `SELECT` | -|---------------------------------------------|-------------------------------------------------------------------------------------------------------------------------|------------------------------| -| `boolean`, `int`, `long`, `float`, `double` | [¿Cómo funciona?)](../sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](../sql-reference/data-types/int-uint.md) | `int` | -| `boolean`, `int`, `long`, `float`, `double` | [Int64](../sql-reference/data-types/int-uint.md), [UInt64](../sql-reference/data-types/int-uint.md) | `long` | -| `boolean`, `int`, `long`, `float`, `double` | [Float32](../sql-reference/data-types/float.md) | `float` | -| `boolean`, `int`, `long`, `float`, `double` | [Float64](../sql-reference/data-types/float.md) | `double` | -| `bytes`, `string`, `fixed`, `enum` | [Cadena](../sql-reference/data-types/string.md) | `bytes` | -| `bytes`, `string`, `fixed` | [Cadena fija (N)](../sql-reference/data-types/fixedstring.md) | `fixed(N)` | -| `enum` | [Enum (8\|16)](../sql-reference/data-types/enum.md) | `enum` | -| `array(T)` | [Matriz (T)](../sql-reference/data-types/array.md) | `array(T)` | -| `union(null, T)`, `union(T, null)` | [Nivel de Cifrado WEP)](../sql-reference/data-types/date.md) | `union(null, T)` | -| `null` | [Nullable (nada)](../sql-reference/data-types/special-data-types/nothing.md) | `null` | -| `int (date)` \* | [Fecha](../sql-reference/data-types/date.md) | `int (date)` \* | -| `long (timestamp-millis)` \* | [¿Qué puedes encontrar en Neodigit)](../sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* | -| `long (timestamp-micros)` \* | [Cómo hacer esto?)](../sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* | - -\* [Tipos lógicos Avro](http://avro.apache.org/docs/current/spec.html#Logical+Types) - -Tipos de datos Avro no admitidos: `record` (no root), `map` - -Tipos de datos lógicos Avro no admitidos: `uuid`, `time-millis`, `time-micros`, `duration` - -### Insertar datos {#inserting-data-1} - -Para insertar datos de un archivo Avro en la tabla ClickHouse: - -``` bash -$ cat file.avro | clickhouse-client --query="INSERT INTO {some_table} FORMAT Avro" -``` - -El esquema raíz del archivo Avro de entrada debe ser de `record` tipo. - -Para encontrar la correspondencia entre las columnas de la tabla y los campos de Avro esquema ClickHouse compara sus nombres. Esta comparación distingue entre mayúsculas y minúsculas. -Los campos no utilizados se omiten. - -Los tipos de datos de las columnas de tabla ClickHouse pueden diferir de los campos correspondientes de los datos de Avro insertados. Al insertar datos, ClickHouse interpreta los tipos de datos de acuerdo con la tabla anterior y luego [elenco](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) los datos al tipo de columna correspondiente. - -### Selección de datos {#selecting-data-1} - -Para seleccionar datos de la tabla ClickHouse en un archivo Avro: - -``` bash -$ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Avro" > file.avro -``` - -Los nombres de columna deben: - -- comenzar con `[A-Za-z_]` -- posteriormente contienen sólo `[A-Za-z0-9_]` - -La compresión de archivos Avro de salida y el intervalo de sincronización se pueden configurar con [Sistema abierto.](../operations/settings/settings.md#settings-output_format_avro_codec) y [Sistema abierto.](../operations/settings/settings.md#settings-output_format_avro_sync_interval) respectivamente. - -## AvroConfluent {#data-format-avro-confluent} - -AvroConfluent admite la decodificación de mensajes Avro de un solo objeto comúnmente utilizados con [Kafka](https://kafka.apache.org/) y [Registro de Esquemas Confluentes](https://docs.confluent.io/current/schema-registry/index.html). - -Cada mensaje de Avro incrusta un id de esquema que se puede resolver en el esquema real con la ayuda del Registro de esquemas. - -Los esquemas se almacenan en caché una vez resueltos. - -La URL del registro de esquemas se configura con [Todos los derechos reservados.](../operations/settings/settings.md#settings-format_avro_schema_registry_url) - -### Coincidencia de tipos de datos {#data_types-matching-1} - -Lo mismo que [Avro](#data-format-avro) - -### Uso {#usage} - -Para verificar rápidamente la resolución del esquema, puede usar [Método de codificación de datos:](https://github.com/edenhill/kafkacat) con [Sistema abierto.](../operations/utilities/clickhouse-local.md#clickhouse-local): - -``` bash -$ kafkacat -b kafka-broker -C -t topic1 -o beginning -f '%s' -c 3 | clickhouse-local --input-format AvroConfluent --format_avro_schema_registry_url 'http://schema-registry' -S "field1 Int64, field2 String" -q 'select * from table' -1 a -2 b -3 c -``` - -Utilizar `AvroConfluent` con [Kafka](../engines/table-engines/integrations/kafka.md): - -``` sql -CREATE TABLE topic1_stream -( - field1 String, - field2 String -) -ENGINE = Kafka() -SETTINGS -kafka_broker_list = 'kafka-broker', -kafka_topic_list = 'topic1', -kafka_group_name = 'group1', -kafka_format = 'AvroConfluent'; - -SET format_avro_schema_registry_url = 'http://schema-registry'; - -SELECT * FROM topic1_stream; -``` - -!!! note "Advertencia" - Configuración `format_avro_schema_registry_url` necesita ser configurado en `users.xml` para mantener su valor después de un reinicio. - -## Parquet {#data-format-parquet} - -[Apache Parquet](http://parquet.apache.org/) es un formato de almacenamiento columnar generalizado en el ecosistema Hadoop. ClickHouse admite operaciones de lectura y escritura para este formato. - -### Coincidencia de tipos de datos {#data_types-matching-2} - -La siguiente tabla muestra los tipos de datos admitidos y cómo coinciden con ClickHouse [tipos de datos](../sql-reference/data-types/index.md) en `INSERT` y `SELECT` consulta. - -| Tipo de datos de parquet (`INSERT`) | Tipo de datos ClickHouse | Tipo de datos de parquet (`SELECT`) | -|-------------------------------------|-----------------------------------------------------------|-------------------------------------| -| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | `UINT8` | -| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | `INT8` | -| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) | `UINT16` | -| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | `INT16` | -| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) | `UINT32` | -| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | `INT32` | -| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | `UINT64` | -| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) | `INT64` | -| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) | `FLOAT` | -| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) | `DOUBLE` | -| `DATE32` | [Fecha](../sql-reference/data-types/date.md) | `UINT16` | -| `DATE64`, `TIMESTAMP` | [FechaHora](../sql-reference/data-types/datetime.md) | `UINT32` | -| `STRING`, `BINARY` | [Cadena](../sql-reference/data-types/string.md) | `STRING` | -| — | [Cadena fija](../sql-reference/data-types/fixedstring.md) | `STRING` | -| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) | `DECIMAL` | - -ClickHouse admite una precisión configurable de `Decimal` tipo. El `INSERT` consulta trata el Parquet `DECIMAL` tipo como el ClickHouse `Decimal128` tipo. - -Tipos de datos de parquet no admitidos: `DATE32`, `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`. - -Los tipos de datos de las columnas de tabla ClickHouse pueden diferir de los campos correspondientes de los datos de Parquet insertados. Al insertar datos, ClickHouse interpreta los tipos de datos de acuerdo con la tabla anterior y luego [elenco](../query_language/functions/type_conversion_functions/#type_conversion_function-cast) los datos de ese tipo de datos que se establece para la columna de tabla ClickHouse. - -### Insertar y seleccionar datos {#inserting-and-selecting-data} - -Puede insertar datos de Parquet desde un archivo en la tabla ClickHouse mediante el siguiente comando: - -``` bash -$ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parquet" -``` - -Puede seleccionar datos de una tabla ClickHouse y guardarlos en algún archivo en el formato Parquet mediante el siguiente comando: - -``` bash -$ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_file.pq} -``` - -Para intercambiar datos con Hadoop, puede usar [Motor de mesa HDFS](../engines/table-engines/integrations/hdfs.md). - -## ORC {#data-format-orc} - -[Apache ORC](https://orc.apache.org/) es un formato de almacenamiento columnar generalizado en el ecosistema Hadoop. Solo puede insertar datos en este formato en ClickHouse. - -### Coincidencia de tipos de datos {#data_types-matching-3} - -La siguiente tabla muestra los tipos de datos admitidos y cómo coinciden con ClickHouse [tipos de datos](../sql-reference/data-types/index.md) en `INSERT` consulta. - -| Tipo de datos ORC (`INSERT`) | Tipo de datos ClickHouse | -|------------------------------|------------------------------------------------------| -| `UINT8`, `BOOL` | [UInt8](../sql-reference/data-types/int-uint.md) | -| `INT8` | [Int8](../sql-reference/data-types/int-uint.md) | -| `UINT16` | [UInt16](../sql-reference/data-types/int-uint.md) | -| `INT16` | [Int16](../sql-reference/data-types/int-uint.md) | -| `UINT32` | [UInt32](../sql-reference/data-types/int-uint.md) | -| `INT32` | [Int32](../sql-reference/data-types/int-uint.md) | -| `UINT64` | [UInt64](../sql-reference/data-types/int-uint.md) | -| `INT64` | [Int64](../sql-reference/data-types/int-uint.md) | -| `FLOAT`, `HALF_FLOAT` | [Float32](../sql-reference/data-types/float.md) | -| `DOUBLE` | [Float64](../sql-reference/data-types/float.md) | -| `DATE32` | [Fecha](../sql-reference/data-types/date.md) | -| `DATE64`, `TIMESTAMP` | [FechaHora](../sql-reference/data-types/datetime.md) | -| `STRING`, `BINARY` | [Cadena](../sql-reference/data-types/string.md) | -| `DECIMAL` | [Decimal](../sql-reference/data-types/decimal.md) | - -ClickHouse soporta la precisión configurable de la `Decimal` tipo. El `INSERT` consulta trata el ORC `DECIMAL` tipo como el ClickHouse `Decimal128` tipo. - -Tipos de datos ORC no admitidos: `DATE32`, `TIME32`, `FIXED_SIZE_BINARY`, `JSON`, `UUID`, `ENUM`. - -Los tipos de datos de las columnas de tabla ClickHouse no tienen que coincidir con los campos de datos ORC correspondientes. Al insertar datos, ClickHouse interpreta los tipos de datos de acuerdo con la tabla anterior y luego [elenco](../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) los datos al tipo de datos establecido para la columna de tabla ClickHouse. - -### Insertar datos {#inserting-data-2} - -Puede insertar datos ORC de un archivo en la tabla ClickHouse mediante el siguiente comando: - -``` bash -$ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT ORC" -``` - -Para intercambiar datos con Hadoop, puede usar [Motor de mesa HDFS](../engines/table-engines/integrations/hdfs.md). - -## Esquema de formato {#formatschema} - -El valor establece el nombre de archivo que contiene el esquema de formato `format_schema`. -Es necesario establecer esta configuración cuando se utiliza uno de los formatos `Cap'n Proto` y `Protobuf`. -El esquema de formato es una combinación de un nombre de archivo y el nombre de un tipo de mensaje en este archivo, delimitado por dos puntos, -e.g. `schemafile.proto:MessageType`. -Si el archivo tiene la extensión estándar para el formato (por ejemplo, `.proto` para `Protobuf`), -se puede omitir y en este caso, el esquema de formato se ve así `schemafile:MessageType`. - -Si introduce o emite datos a través del [cliente](../interfaces/cli.md) en el [modo interactivo](../interfaces/cli.md#cli_usage), el nombre de archivo especificado en el esquema de formato -puede contener una ruta absoluta o una ruta relativa al directorio actual en el cliente. -Si utiliza el cliente en el [modo por lotes](../interfaces/cli.md#cli_usage), la ruta de acceso al esquema debe ser relativa por razones de seguridad. - -Si introduce o emite datos a través del [Interfaz HTTP](../interfaces/http.md) el nombre de archivo especificado en el esquema de formato -debe estar ubicado en el directorio especificado en [format_schema_path](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-format_schema_path) -en la configuración del servidor. - -## Salto de errores {#skippingerrors} - -Algunos formatos como `CSV`, `TabSeparated`, `TSKV`, `JSONEachRow`, `Template`, `CustomSeparated` y `Protobuf` puede omitir la fila rota si se produjo un error de análisis y continuar el análisis desde el comienzo de la siguiente fila. Ver [Entrada_format_allow_errors_num](../operations/settings/settings.md#settings-input_format_allow_errors_num) y -[Entrada_format_allow_errors_ratio](../operations/settings/settings.md#settings-input_format_allow_errors_ratio) configuración. -Limitacion: -- En caso de error de análisis `JSONEachRow` omite todos los datos hasta la nueva línea (o EOF), por lo que las filas deben estar delimitadas por `\n` para contar los errores correctamente. -- `Template` y `CustomSeparated` use el delimitador después de la última columna y el delimitador entre filas para encontrar el comienzo de la siguiente fila, por lo que omitir errores solo funciona si al menos uno de ellos no está vacío. - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/formats/) diff --git a/docs/es/interfaces/http.md b/docs/es/interfaces/http.md deleted file mode 100644 index ab510a268e3..00000000000 --- a/docs/es/interfaces/http.md +++ /dev/null @@ -1,617 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 19 -toc_title: Interfaz HTTP ---- - -# Interfaz HTTP {#http-interface} - -La interfaz HTTP le permite usar ClickHouse en cualquier plataforma desde cualquier lenguaje de programación. Lo usamos para trabajar desde Java y Perl, así como scripts de shell. En otros departamentos, la interfaz HTTP se usa desde Perl, Python y Go. La interfaz HTTP es más limitada que la interfaz nativa, pero tiene una mejor compatibilidad. - -De forma predeterminada, clickhouse-server escucha HTTP en el puerto 8123 (esto se puede cambiar en la configuración). - -Si realiza una solicitud GET / sin parámetros, devuelve 200 códigos de respuesta y la cadena que definió en [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) valor predeterminado “Ok.” (con un avance de línea al final) - -``` bash -$ curl 'http://localhost:8123/' -Ok. -``` - -Use la solicitud GET / ping en los scripts de comprobación de estado. Este controlador siempre devuelve “Ok.” (con un avance de línea al final). Disponible a partir de la versión 18.12.13. - -``` bash -$ curl 'http://localhost:8123/ping' -Ok. -``` - -Enviar la solicitud como una URL ‘query’ parámetro, o como un POST. O envíe el comienzo de la consulta en el ‘query’ parámetro, y el resto en el POST (explicaremos más adelante por qué esto es necesario). El tamaño de la URL está limitado a 16 KB, así que tenga esto en cuenta al enviar consultas grandes. - -Si tiene éxito, recibirá el código de respuesta 200 y el resultado en el cuerpo de respuesta. -Si se produce un error, recibirá el código de respuesta 500 y un texto de descripción de error en el cuerpo de la respuesta. - -Al usar el método GET, ‘readonly’ se establece. En otras palabras, para consultas que modifican datos, solo puede usar el método POST. Puede enviar la consulta en sí misma en el cuerpo POST o en el parámetro URL. - -Ejemplos: - -``` bash -$ curl 'http://localhost:8123/?query=SELECT%201' -1 - -$ wget -nv -O- 'http://localhost:8123/?query=SELECT 1' -1 - -$ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 -HTTP/1.0 200 OK -Date: Wed, 27 Nov 2019 10:30:18 GMT -Connection: Close -Content-Type: text/tab-separated-values; charset=UTF-8 -X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal -X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} - -1 -``` - -Como puede ver, curl es algo inconveniente ya que los espacios deben ser URL escapadas. -Aunque wget escapa de todo en sí, no recomendamos usarlo porque no funciona bien sobre HTTP 1.1 cuando se usa keep-alive y Transfer-Encoding: chunked . - -``` bash -$ echo 'SELECT 1' | curl 'http://localhost:8123/' --data-binary @- -1 - -$ echo 'SELECT 1' | curl 'http://localhost:8123/?query=' --data-binary @- -1 - -$ echo '1' | curl 'http://localhost:8123/?query=SELECT' --data-binary @- -1 -``` - -Si se envía parte de la consulta en el parámetro y parte en el POST, se inserta un avance de línea entre estas dos partes de datos. -Ejemplo (esto no funcionará): - -``` bash -$ echo 'ECT 1' | curl 'http://localhost:8123/?query=SEL' --data-binary @- -Code: 59, e.displayText() = DB::Exception: Syntax error: failed at position 0: SEL -ECT 1 -, expected One of: SHOW TABLES, SHOW DATABASES, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE., e.what() = DB::Exception -``` - -De forma predeterminada, los datos se devuelven en formato TabSeparated (para obtener más información, “Formats” apartado). -Utilice la cláusula FORMAT de la consulta para solicitar cualquier otro formato. - -``` bash -$ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @- -┏━━━┓ -┃ 1 ┃ -┡━━━┩ -│ 1 │ -└───┘ -``` - -El método POST de transmitir datos es necesario para las consultas INSERT. En este caso, puede escribir el comienzo de la consulta en el parámetro URL y usar POST para pasar los datos a insertar. Los datos a insertar podrían ser, por ejemplo, un volcado separado por tabuladores de MySQL. De esta manera, la consulta INSERT reemplaza LOAD DATA LOCAL INFILE de MySQL. - -Ejemplos: Crear una tabla: - -``` bash -$ echo 'CREATE TABLE t (a UInt8) ENGINE = Memory' | curl 'http://localhost:8123/' --data-binary @- -``` - -Uso de la consulta INSERT familiar para la inserción de datos: - -``` bash -$ echo 'INSERT INTO t VALUES (1),(2),(3)' | curl 'http://localhost:8123/' --data-binary @- -``` - -Los datos se pueden enviar por separado de la consulta: - -``` bash -$ echo '(4),(5),(6)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20VALUES' --data-binary @- -``` - -Puede especificar cualquier formato de datos. El ‘Values’ el formato es el mismo que el que se usa al escribir INSERT INTO t VALUES: - -``` bash -$ echo '(7),(8),(9)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20FORMAT%20Values' --data-binary @- -``` - -Para insertar datos de un volcado separado por tabuladores, especifique el formato correspondiente: - -``` bash -$ echo -ne '10\n11\n12\n' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20FORMAT%20TabSeparated' --data-binary @- -``` - -Lectura del contenido de la tabla. Los datos se emiten en orden aleatorio debido al procesamiento de consultas paralelas: - -``` bash -$ curl 'http://localhost:8123/?query=SELECT%20a%20FROM%20t' -7 -8 -9 -10 -11 -12 -1 -2 -3 -4 -5 -6 -``` - -Eliminando la mesa. - -``` bash -$ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- -``` - -Para las solicitudes correctas que no devuelven una tabla de datos, se devuelve un cuerpo de respuesta vacío. - -Puede utilizar el formato interno de compresión ClickHouse al transmitir datos. Los datos comprimidos tienen un formato no estándar, y deberá usar el `clickhouse-compressor` programa para trabajar con él (se instala con el `clickhouse-client` paquete). Para aumentar la eficiencia de la inserción de datos, puede deshabilitar la verificación de suma de comprobación [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) configuración. - -Si ha especificado `compress=1` en la URL, el servidor comprime los datos que le envía. -Si ha especificado `decompress=1` en la dirección URL, el servidor descomprime los mismos datos que `POST` método. - -También puede optar por utilizar [Compresión HTTP](https://en.wikipedia.org/wiki/HTTP_compression). Para enviar un `POST` solicitud, agregue el encabezado de solicitud `Content-Encoding: compression_method`. Para que ClickHouse comprima la respuesta, debe agregar `Accept-Encoding: compression_method`. Soporta ClickHouse `gzip`, `br`, y `deflate` [métodos de compresión](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). Para habilitar la compresión HTTP, debe usar ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) configuración. Puede configurar el nivel de compresión de datos [http_zlib_compression_level](#settings-http_zlib_compression_level) para todos los métodos de compresión. - -Puede usar esto para reducir el tráfico de red al transmitir una gran cantidad de datos o para crear volcados que se comprimen inmediatamente. - -Ejemplos de envío de datos con compresión: - -``` bash -#Sending data to the server: -$ curl -vsS "http://localhost:8123/?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' -H 'Accept-Encoding: gzip' - -#Sending data to the client: -$ echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' -``` - -!!! note "Nota" - Algunos clientes HTTP pueden descomprimir datos del servidor de forma predeterminada (con `gzip` y `deflate`) y puede obtener datos descomprimidos incluso si usa la configuración de compresión correctamente. - -Puede usar el ‘database’ Parámetro URL para especificar la base de datos predeterminada. - -``` bash -$ echo 'SELECT number FROM numbers LIMIT 10' | curl 'http://localhost:8123/?database=system' --data-binary @- -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -``` - -De forma predeterminada, la base de datos que está registrada en la configuración del servidor se utiliza como base de datos predeterminada. De forma predeterminada, esta es la base de datos llamada ‘default’. Como alternativa, siempre puede especificar la base de datos utilizando un punto antes del nombre de la tabla. - -El nombre de usuario y la contraseña se pueden indicar de una de estas tres maneras: - -1. Uso de la autenticación básica HTTP. Ejemplo: - - - -``` bash -$ echo 'SELECT 1' | curl 'http://user:password@localhost:8123/' -d @- -``` - -1. En el ‘user’ y ‘password’ Parámetros de URL. Ejemplo: - - - -``` bash -$ echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @- -``` - -1. Utilizar ‘X-ClickHouse-User’ y ‘X-ClickHouse-Key’ cabecera. Ejemplo: - - - -``` bash -$ echo 'SELECT 1' | curl -H 'X-ClickHouse-User: user' -H 'X-ClickHouse-Key: password' 'http://localhost:8123/' -d @- -``` - -Si no se especifica el nombre de usuario, `default` se utiliza el nombre. Si no se especifica la contraseña, se utiliza la contraseña vacía. -También puede utilizar los parámetros de URL para especificar cualquier configuración para procesar una sola consulta o perfiles completos de configuración. Ejemplo:http://localhost:8123/?perfil=web&max_rows_to_read=1000000000&consulta=SELECCIONA+1 - -Para obtener más información, consulte [Configuración](../operations/settings/index.md) apartado. - -``` bash -$ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:8123/?' --data-binary @- -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -``` - -Para obtener información sobre otros parámetros, consulte la sección “SET”. - -Del mismo modo, puede utilizar sesiones ClickHouse en el protocolo HTTP. Para hacer esto, debe agregar el `session_id` GET parámetro a la solicitud. Puede usar cualquier cadena como ID de sesión. De forma predeterminada, la sesión finaliza después de 60 segundos de inactividad. Para cambiar este tiempo de espera, modifique `default_session_timeout` configuración en la configuración del servidor, o `session_timeout` GET parámetro a la solicitud. Para comprobar el estado de la sesión, `session_check=1` parámetro. Solo se puede ejecutar una consulta a la vez en una sola sesión. - -Puede recibir información sobre el progreso de una consulta en `X-ClickHouse-Progress` encabezados de respuesta. Para hacer esto, habilite [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Ejemplo de la secuencia de encabezado: - -``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} -``` - -Posibles campos de encabezado: - -- `read_rows` — Number of rows read. -- `read_bytes` — Volume of data read in bytes. -- `total_rows_to_read` — Total number of rows to be read. -- `written_rows` — Number of rows written. -- `written_bytes` — Volume of data written in bytes. - -Las solicitudes en ejecución no se detienen automáticamente si se pierde la conexión HTTP. El análisis y el formato de datos se realizan en el lado del servidor, y el uso de la red puede ser ineficaz. -Opcional ‘query_id’ parámetro se puede pasar como el ID de consulta (cualquier cadena). Para obtener más información, consulte la sección “Settings, replace_running_query”. - -Opcional ‘quota_key’ parámetro se puede pasar como la clave de cuota (cualquier cadena). Para obtener más información, consulte la sección “Quotas”. - -La interfaz HTTP permite pasar datos externos (tablas temporales externas) para consultar. Para obtener más información, consulte la sección “External data for query processing”. - -## Almacenamiento en búfer de respuesta {#response-buffering} - -Puede habilitar el almacenamiento en búfer de respuestas en el lado del servidor. El `buffer_size` y `wait_end_of_query` Los parámetros URL se proporcionan para este propósito. - -`buffer_size` determina el número de bytes en el resultado para almacenar en búfer en la memoria del servidor. Si un cuerpo de resultado es mayor que este umbral, el búfer se escribe en el canal HTTP y los datos restantes se envían directamente al canal HTTP. - -Para asegurarse de que toda la respuesta se almacena en búfer, establezca `wait_end_of_query=1`. En este caso, los datos que no se almacenan en la memoria se almacenarán en un archivo de servidor temporal. - -Ejemplo: - -``` bash -$ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wait_end_of_query=1' -d 'SELECT toUInt8(number) FROM system.numbers LIMIT 9000000 FORMAT RowBinary' -``` - -Utilice el almacenamiento en búfer para evitar situaciones en las que se produjo un error de procesamiento de consultas después de enviar al cliente el código de respuesta y los encabezados HTTP. En esta situación, se escribe un mensaje de error al final del cuerpo de la respuesta y, en el lado del cliente, el error solo se puede detectar en la etapa de análisis. - -### Consultas con parámetros {#cli-queries-with-parameters} - -Puede crear una consulta con parámetros y pasar valores para ellos desde los parámetros de solicitud HTTP correspondientes. Para obtener más información, consulte [Consultas con parámetros para CLI](cli.md#cli-queries-with-parameters). - -### Ejemplo {#example} - -``` bash -$ curl -sS "

?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}" -``` - -## Interfaz HTTP predefinida {#predefined_http_interface} - -ClickHouse admite consultas específicas a través de la interfaz HTTP. Por ejemplo, puede escribir datos en una tabla de la siguiente manera: - -``` bash -$ echo '(4),(5),(6)' | curl 'http://localhost:8123/?query=INSERT%20INTO%20t%20VALUES' --data-binary @- -``` - -ClickHouse también es compatible con la interfaz HTTP predefinida que puede ayudarle a una integración más fácil con herramientas de terceros como [Prometheus exportador](https://github.com/percona-lab/clickhouse_exporter). - -Ejemplo: - -- En primer lugar, agregue esta sección al archivo de configuración del servidor: - - - -``` xml - - - /predefined_query - POST,GET - - predefined_query_handler - SELECT * FROM system.metrics LIMIT 5 FORMAT Template SETTINGS format_template_resultset = 'prometheus_template_output_format_resultset', format_template_row = 'prometheus_template_output_format_row', format_template_rows_between_delimiter = '\n' - - - ... - ... - -``` - -- Ahora puede solicitar la url directamente para los datos en el formato Prometheus: - - - -``` bash -$ curl -v 'http://localhost:8123/predefined_query' -* Trying ::1... -* Connected to localhost (::1) port 8123 (#0) -> GET /predefined_query HTTP/1.1 -> Host: localhost:8123 -> User-Agent: curl/7.47.0 -> Accept: */* -> -< HTTP/1.1 200 OK -< Date: Tue, 28 Apr 2020 08:52:56 GMT -< Connection: Keep-Alive -< Content-Type: text/plain; charset=UTF-8 -< X-ClickHouse-Server-Display-Name: i-mloy5trc -< Transfer-Encoding: chunked -< X-ClickHouse-Query-Id: 96fe0052-01e6-43ce-b12a-6b7370de6e8a -< X-ClickHouse-Format: Template -< X-ClickHouse-Timezone: Asia/Shanghai -< Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< -# HELP "Query" "Number of executing queries" -# TYPE "Query" counter -"Query" 1 - -# HELP "Merge" "Number of executing background merges" -# TYPE "Merge" counter -"Merge" 0 - -# HELP "PartMutation" "Number of mutations (ALTER DELETE/UPDATE)" -# TYPE "PartMutation" counter -"PartMutation" 0 - -# HELP "ReplicatedFetch" "Number of data parts being fetched from replica" -# TYPE "ReplicatedFetch" counter -"ReplicatedFetch" 0 - -# HELP "ReplicatedSend" "Number of data parts being sent to replicas" -# TYPE "ReplicatedSend" counter -"ReplicatedSend" 0 - -* Connection #0 to host localhost left intact - - -* Connection #0 to host localhost left intact -``` - -Como puede ver en el ejemplo, si `` está configurado en la configuración.archivo xml y `` puede contener muchos `s`. ClickHouse coincidirá con las solicitudes HTTP recibidas con el tipo predefinido en `` y el primer emparejado ejecuta el controlador. Luego, ClickHouse ejecutará la consulta predefinida correspondiente si la coincidencia es exitosa. - -> Ahora `` puede configurar ``, ``, ``,``: -> `` es responsable de hacer coincidir la parte del método de la solicitud HTTP. `` se ajusta plenamente a la definición de [método](https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods) en el protocolo HTTP. Es una configuración opcional. Si no está definido en el archivo de configuración, no coincide con la parte del método de la solicitud HTTP. -> -> `` es responsable de hacer coincidir la parte url de la solicitud HTTP. Es compatible con [RE2](https://github.com/google/re2)expresiones regulares. Es una configuración opcional. Si no está definido en el archivo de configuración, no coincide con la parte url de la solicitud HTTP. -> -> `` es responsable de hacer coincidir la parte del encabezado de la solicitud HTTP. Es compatible con las expresiones regulares de RE2. Es una configuración opcional. Si no está definido en el archivo de configuración, no coincide con la parte de encabezado de la solicitud HTTP. -> -> `` contiene la parte de procesamiento principal. Ahora `` puede configurar ``, ``, ``, ``, ``, ``. -> \> `` Actualmente soporta tres tipos: **Dirección de correo electrónico**, **Nombre de la red inalámbrica (SSID):**, **estática**. -> \> -> \> `` - utilizar con el tipo predefined_query_handler, ejecuta la consulta cuando se llama al controlador. -> \> -> \> `` - utilizar con el tipo dynamic_query_handler, extrae y ejecuta el valor correspondiente al `` valor en parámetros de solicitud HTTP. -> \> -> \> `` - uso con tipo estático, código de estado de respuesta. -> \> -> \> `` - uso con tipo estático, respuesta [tipo de contenido](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). -> \> -> \> `` - uso con tipo estático, contenido de respuesta enviado al cliente, cuando se usa el prefijo ‘file://’ o ‘config://’, encontrar el contenido del archivo o configuración enviar al cliente. - -A continuación están los métodos de configuración para los diferentes ``. - -## Dirección de correo electrónico {#predefined_query_handler} - -`` admite la configuración de valores Settings y query_params. Puede configurar `` en el tipo de ``. - -`` valor es una consulta predefinida de ``, que es ejecutado por ClickHouse cuando se hace coincidir una solicitud HTTP y se devuelve el resultado de la consulta. Es una configuración imprescindible. - -En el ejemplo siguiente se definen los valores de `max_threads` y `max_alter_threads` configuración, a continuación, consulta la tabla del sistema para comprobar si estos ajustes se han establecido correctamente. - -Ejemplo: - -``` xml - - - [^/]+)(/(?P[^/]+))?]]> - GET - - TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> - - - predefined_query_handler - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} - - - -``` - -``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_alter_threads?max_threads=1&max_alter_threads=2' -1 -max_alter_threads 2 -``` - -!!! note "precaución" - En uno `` sólo es compatible con uno `` de un tipo de plaquita. - -## Nombre de la red inalámbrica (SSID): {#dynamic_query_handler} - -En ``, consulta se escribe en forma de param de la solicitud HTTP. La diferencia es que en ``, consulta se escribe en el archivo de configuración. Puede configurar `` en ``. - -ClickHouse extrae y ejecuta el valor correspondiente al `` valor en la url de la solicitud HTTP. El valor predeterminado de `` ser `/query` . Es una configuración opcional. Si no hay una definición en el archivo de configuración, el parámetro no se pasa. - -Para experimentar con esta funcionalidad, el ejemplo define los valores de max_threads y max_alter_threads y consulta si la configuración se estableció correctamente. - -Ejemplo: - -``` xml - - - - TEST_HEADER_VALUE_DYNAMIC - - dynamic_query_handler - query_param - - - -``` - -``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE_DYNAMIC' 'http://localhost:8123/own?max_threads=1&max_alter_threads=2¶m_name_1=max_threads¶m_name_2=max_alter_threads&query_param=SELECT%20name,value%20FROM%20system.settings%20where%20name%20=%20%7Bname_1:String%7D%20OR%20name%20=%20%7Bname_2:String%7D' -max_threads 1 -max_alter_threads 2 -``` - -## estática {#static} - -`` puede volver [Content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [estatus](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) y response_content. response_content puede devolver el contenido especificado - -Ejemplo: - -Devuelve un mensaje. - -``` xml - - - GET - xxx - /hi - - static - 402 - text/html; charset=UTF-8 - Say Hi! - - - -``` - -``` bash -$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' -* Trying ::1... -* Connected to localhost (::1) port 8123 (#0) -> GET /hi HTTP/1.1 -> Host: localhost:8123 -> User-Agent: curl/7.47.0 -> Accept: */* -> XXX:xxx -> -< HTTP/1.1 402 Payment Required -< Date: Wed, 29 Apr 2020 03:51:26 GMT -< Connection: Keep-Alive -< Content-Type: text/html; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< -* Connection #0 to host localhost left intact -Say Hi!% -``` - -Busque el contenido de la configuración enviada al cliente. - -``` xml -
]]>
- - - - GET - xxx - /get_config_static_handler - - static - config://get_config_static_handler - - - -``` - -``` bash -$ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' -* Trying ::1... -* Connected to localhost (::1) port 8123 (#0) -> GET /get_config_static_handler HTTP/1.1 -> Host: localhost:8123 -> User-Agent: curl/7.47.0 -> Accept: */* -> XXX:xxx -> -< HTTP/1.1 200 OK -< Date: Wed, 29 Apr 2020 04:01:24 GMT -< Connection: Keep-Alive -< Content-Type: text/plain; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< -* Connection #0 to host localhost left intact -
% -``` - -Encuentra el contenido del archivo enviado al cliente. - -``` xml - - - GET - xxx - /get_absolute_path_static_handler - - static - text/html; charset=UTF-8 - file:///absolute_path_file.html - - - - GET - xxx - /get_relative_path_static_handler - - static - text/html; charset=UTF-8 - file://./relative_path_file.html - - - -``` - -``` bash -$ user_files_path='/var/lib/clickhouse/user_files' -$ sudo echo "Relative Path File" > $user_files_path/relative_path_file.html -$ sudo echo "Absolute Path File" > $user_files_path/absolute_path_file.html -$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' -* Trying ::1... -* Connected to localhost (::1) port 8123 (#0) -> GET /get_absolute_path_static_handler HTTP/1.1 -> Host: localhost:8123 -> User-Agent: curl/7.47.0 -> Accept: */* -> XXX:xxx -> -< HTTP/1.1 200 OK -< Date: Wed, 29 Apr 2020 04:18:16 GMT -< Connection: Keep-Alive -< Content-Type: text/html; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< -Absolute Path File -* Connection #0 to host localhost left intact -$ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' -* Trying ::1... -* Connected to localhost (::1) port 8123 (#0) -> GET /get_relative_path_static_handler HTTP/1.1 -> Host: localhost:8123 -> User-Agent: curl/7.47.0 -> Accept: */* -> XXX:xxx -> -< HTTP/1.1 200 OK -< Date: Wed, 29 Apr 2020 04:18:31 GMT -< Connection: Keep-Alive -< Content-Type: text/html; charset=UTF-8 -< Transfer-Encoding: chunked -< Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} -< -Relative Path File -* Connection #0 to host localhost left intact -``` - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/http_interface/) diff --git a/docs/es/interfaces/index.md b/docs/es/interfaces/index.md deleted file mode 100644 index 3632c8a9e29..00000000000 --- a/docs/es/interfaces/index.md +++ /dev/null @@ -1,29 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Interfaz -toc_priority: 14 -toc_title: "Implantaci\xF3n" ---- - -# Interfaz {#interfaces} - -ClickHouse proporciona dos interfaces de red (ambas se pueden ajustar opcionalmente en TLS para mayor seguridad): - -- [HTTP](http.md), que está documentado y fácil de usar directamente. -- [TCP nativo](tcp.md), que tiene menos sobrecarga. - -En la mayoría de los casos, se recomienda utilizar la herramienta o biblioteca apropiada en lugar de interactuar con ellos directamente. Oficialmente apoyados por Yandex son los siguientes: - -- [Cliente de línea de comandos](cli.md) -- [Controlador JDBC](jdbc.md) -- [Controlador ODBC](odbc.md) -- [Biblioteca cliente de C++](cpp.md) - -También hay una amplia gama de bibliotecas de terceros para trabajar con ClickHouse: - -- [Bibliotecas de clientes](third-party/client-libraries.md) -- [Integración](third-party/integrations.md) -- [Interfaces visuales](third-party/gui.md) - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/) diff --git a/docs/es/interfaces/jdbc.md b/docs/es/interfaces/jdbc.md deleted file mode 100644 index 7303dec8960..00000000000 --- a/docs/es/interfaces/jdbc.md +++ /dev/null @@ -1,15 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 22 -toc_title: Controlador JDBC ---- - -# Controlador JDBC {#jdbc-driver} - -- **[Conductor oficial](https://github.com/ClickHouse/clickhouse-jdbc)** -- Controladores de terceros: - - [Sistema abierto.](https://github.com/housepower/ClickHouse-Native-JDBC) - - [Método de codificación de datos:](https://github.com/blynkkk/clickhouse4j) - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/jdbc/) diff --git a/docs/es/interfaces/mysql.md b/docs/es/interfaces/mysql.md deleted file mode 100644 index a5124c61dd5..00000000000 --- a/docs/es/interfaces/mysql.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 20 -toc_title: Interfaz MySQL ---- - -# Interfaz MySQL {#mysql-interface} - -ClickHouse soporta el protocolo de cable MySQL. Puede ser habilitado por [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) configuración en el archivo de configuración: - -``` xml -9004 -``` - -Ejemplo de conexión mediante la herramienta de línea de comandos `mysql`: - -``` bash -$ mysql --protocol tcp -u default -P 9004 -``` - -Salida si una conexión se realizó correctamente: - -``` text -Welcome to the MySQL monitor. Commands end with ; or \g. -Your MySQL connection id is 4 -Server version: 20.2.1.1-ClickHouse - -Copyright (c) 2000, 2019, Oracle and/or its affiliates. All rights reserved. - -Oracle is a registered trademark of Oracle Corporation and/or its -affiliates. Other names may be trademarks of their respective -owners. - -Type 'help;' or '\h' for help. Type '\c' to clear the current input statement. - -mysql> -``` - -Para la compatibilidad con todos los clientes MySQL, se recomienda especificar la contraseña de usuario con [doble SHA1](../operations/settings/settings-users.md#password_double_sha1_hex) en el archivo de configuración. -Si la contraseña de usuario se especifica usando [SHA256](../operations/settings/settings-users.md#password_sha256_hex), algunos clientes no podrán autenticarse (mysqljs y versiones antiguas de la herramienta de línea de comandos mysql). - -Restricción: - -- las consultas preparadas no son compatibles - -- algunos tipos de datos se envían como cadenas - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/mysql/) diff --git a/docs/es/interfaces/odbc.md b/docs/es/interfaces/odbc.md deleted file mode 100644 index 6ccb979c7f7..00000000000 --- a/docs/es/interfaces/odbc.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 23 -toc_title: Conductor ODBC ---- - -# Conductor ODBC {#odbc-driver} - -- [Conductor oficial](https://github.com/ClickHouse/clickhouse-odbc). - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/odbc/) diff --git a/docs/es/interfaces/tcp.md b/docs/es/interfaces/tcp.md deleted file mode 100644 index 47df0d12829..00000000000 --- a/docs/es/interfaces/tcp.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 18 -toc_title: Interfaz nativa (TCP) ---- - -# Interfaz nativa (TCP) {#native-interface-tcp} - -El protocolo nativo se utiliza en el [cliente de línea de comandos](cli.md), para la comunicación entre servidores durante el procesamiento de consultas distribuidas, y también en otros programas de C, Desafortunadamente, el protocolo nativo de ClickHouse aún no tiene especificaciones formales, pero puede ser diseñado de manera inversa desde el código fuente de ClickHouse (comenzando [por aquí](https://github.com/ClickHouse/ClickHouse/tree/master/src/Client)) y/o mediante la interceptación y el análisis del tráfico TCP. - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/tcp/) diff --git a/docs/es/interfaces/third-party/client-libraries.md b/docs/es/interfaces/third-party/client-libraries.md deleted file mode 100644 index b61ab1a5d9c..00000000000 --- a/docs/es/interfaces/third-party/client-libraries.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -toc_priority: 26 -toc_title: Client Libraries ---- - -# Client Libraries from Third-party Developers {#client-libraries-from-third-party-developers} - -!!! warning "Disclaimer" - Yandex does **not** maintain the libraries listed below and haven’t done any extensive testing to ensure their quality. - -- Python - - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) - - [clickhouse-client](https://github.com/yurial/clickhouse-client) - - [aiochclient](https://github.com/maximdanilchenko/aiochclient) - - [asynch](https://github.com/long2ice/asynch) -- PHP - - [smi2/phpclickhouse](https://packagist.org/packages/smi2/phpClickHouse) - - [8bitov/clickhouse-php-client](https://packagist.org/packages/8bitov/clickhouse-php-client) - - [bozerkins/clickhouse-client](https://packagist.org/packages/bozerkins/clickhouse-client) - - [simpod/clickhouse-client](https://packagist.org/packages/simpod/clickhouse-client) - - [seva-code/php-click-house-client](https://packagist.org/packages/seva-code/php-click-house-client) - - [SeasClick C++ client](https://github.com/SeasX/SeasClick) -- Go - - [clickhouse](https://github.com/kshvakov/clickhouse/) - - [go-clickhouse](https://github.com/roistat/go-clickhouse) - - [mailrugo-clickhouse](https://github.com/mailru/go-clickhouse) - - [golang-clickhouse](https://github.com/leprosus/golang-clickhouse) -- NodeJs - - [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse) - - [node-clickhouse](https://github.com/apla/node-clickhouse) -- Perl - - [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse) - - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) - - [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse) -- Ruby - - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - - [clickhouse-activerecord](https://github.com/PNixx/clickhouse-activerecord) -- R - - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - - [RClickHouse](https://github.com/IMSMWU/RClickHouse) -- Java - - [clickhouse-client-java](https://github.com/VirtusAI/clickhouse-client-java) - - [clickhouse-client](https://github.com/Ecwid/clickhouse-client) -- Scala - - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- Kotlin - - [AORM](https://github.com/TanVD/AORM) -- C# - - [Octonica.ClickHouseClient](https://github.com/Octonica/ClickHouseClient) - - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) - - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) -- Elixir - - [clickhousex](https://github.com/appodeal/clickhousex/) - - [pillar](https://github.com/sofakingworld/pillar) -- Nim - - [nim-clickhouse](https://github.com/leonardoce/nim-clickhouse) - -[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/client_libraries/) diff --git a/docs/es/interfaces/third-party/gui.md b/docs/es/interfaces/third-party/gui.md deleted file mode 100644 index 754c0f68c69..00000000000 --- a/docs/es/interfaces/third-party/gui.md +++ /dev/null @@ -1,156 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 28 -toc_title: Interfaces Visuales ---- - -# Interfaces visuales de desarrolladores de terceros {#visual-interfaces-from-third-party-developers} - -## De código abierto {#open-source} - -### Tabix {#tabix} - -Interfaz web para ClickHouse en el [Tabix](https://github.com/tabixio/tabix) proyecto. - -Función: - -- Funciona con ClickHouse directamente desde el navegador, sin la necesidad de instalar software adicional. -- Editor de consultas con resaltado de sintaxis. -- Autocompletado de comandos. -- Herramientas para el análisis gráfico de la ejecución de consultas. -- Opciones de esquema de color. - -[Documentación de Tabix](https://tabix.io/doc/). - -### Sistema abierto {#houseops} - -[Sistema abierto.](https://github.com/HouseOps/HouseOps) Es una interfaz de usuario / IDE para OSX, Linux y Windows. - -Función: - -- Generador de consultas con resaltado de sintaxis. Ver la respuesta en una tabla o vista JSON. -- Exportar resultados de consultas como CSV o JSON. -- Lista de procesos con descripciones. Modo de escritura. Capacidad de parar (`KILL`) proceso. -- Gráfico de base de datos. Muestra todas las tablas y sus columnas con información adicional. -- Una vista rápida del tamaño de la columna. -- Configuración del servidor. - -Las siguientes características están planificadas para el desarrollo: - -- Gestión de bases de datos. -- Gestión de usuarios. -- Análisis de datos en tiempo real. -- Supervisión de clúster. -- Gestión de clústeres. -- Monitoreo de tablas replicadas y Kafka. - -### Faro {#lighthouse} - -[Faro](https://github.com/VKCOM/lighthouse) Es una interfaz web ligera para ClickHouse. - -Función: - -- Lista de tablas con filtrado y metadatos. -- Vista previa de la tabla con filtrado y clasificación. -- Ejecución de consultas de sólo lectura. - -### Redash {#redash} - -[Redash](https://github.com/getredash/redash) es una plataforma para la visualización de datos. - -Admite múltiples fuentes de datos, incluido ClickHouse, Redash puede unir los resultados de consultas de diferentes fuentes de datos en un conjunto de datos final. - -Función: - -- Potente editor de consultas. -- Explorador de base de datos. -- Herramientas de visualización, que le permiten representar datos en diferentes formas. - -### DBeaver {#dbeaver} - -[DBeaver](https://dbeaver.io/) - Cliente de base de datos de escritorio universal con soporte ClickHouse. - -Función: - -- Desarrollo de consultas con resaltado de sintaxis y autocompletado. -- Lista de tablas con filtros y búsqueda de metadatos. -- Vista previa de datos de tabla. -- Búsqueda de texto completo. - -### Sistema abierto {#clickhouse-cli} - -[Sistema abierto.](https://github.com/hatarist/clickhouse-cli) es un cliente de línea de comandos alternativo para ClickHouse, escrito en Python 3. - -Función: - -- Autocompletado. -- Resaltado de sintaxis para las consultas y la salida de datos. -- Soporte de buscapersonas para la salida de datos. -- Comandos similares a PostgreSQL personalizados. - -### Sistema abierto {#clickhouse-flamegraph} - -[Sistema abierto.](https://github.com/Slach/clickhouse-flamegraph) es una herramienta especializada para visualizar el `system.trace_log` como [Flamegraph](http://www.brendangregg.com/flamegraphs.html). - -### Bienvenidos al Portal de Licitación Electrónica de Licitación Electrónica {#clickhouse-plantuml} - -[Método de codificación de datos:](https://pypi.org/project/clickhouse-plantuml/) es un script para generar [PlantUML](https://plantuml.com/) diagrama de esquemas de tablas. - -## Comercial {#commercial} - -### DataGrip {#datagrip} - -[DataGrip](https://www.jetbrains.com/datagrip/) Es un IDE de base de datos de JetBrains con soporte dedicado para ClickHouse. También está integrado en otras herramientas basadas en IntelliJ: PyCharm, IntelliJ IDEA, GoLand, PhpStorm y otros. - -Función: - -- Finalización de código muy rápida. -- Resaltado de sintaxis de ClickHouse. -- Soporte para características específicas de ClickHouse, por ejemplo, columnas anidadas, motores de tablas. -- Editor de datos. -- Refactorizaciones. -- Búsqueda y navegación. - -### Yandex DataLens {#yandex-datalens} - -[Yandex DataLens](https://cloud.yandex.ru/services/datalens) es un servicio de visualización y análisis de datos. - -Función: - -- Amplia gama de visualizaciones disponibles, desde simples gráficos de barras hasta paneles complejos. -- Los paneles podrían ponerse a disposición del público. -- Soporte para múltiples fuentes de datos, incluyendo ClickHouse. -- Almacenamiento de datos materializados basados en ClickHouse. - -Nivel de Cifrado WEP [disponible de forma gratuita](https://cloud.yandex.com/docs/datalens/pricing) para proyectos de baja carga, incluso para uso comercial. - -- [Documentación de DataLens](https://cloud.yandex.com/docs/datalens/). -- [Tutorial](https://cloud.yandex.com/docs/solutions/datalens/data-from-ch-visualization) en la visualización de datos de una base de datos ClickHouse. - -### Software de Holística {#holistics-software} - -[Holística](https://www.holistics.io/) es una plataforma de datos de pila completa y una herramienta de inteligencia de negocios. - -Función: - -- Correo electrónico automatizado, Slack y horarios de informes de Google Sheet. -- Editor SQL con visualizaciones, control de versiones, autocompletado, componentes de consulta reutilizables y filtros dinámicos. -- Análisis integrado de informes y cuadros de mando a través de iframe. -- Preparación de datos y capacidades ETL. -- Soporte de modelado de datos SQL para mapeo relacional de datos. - -### Mirador {#looker} - -[Mirador](https://looker.com) Es una plataforma de datos y una herramienta de inteligencia de negocios con soporte para más de 50 dialectos de bases de datos, incluido ClickHouse. Bravo está disponible como una plataforma SaaS y auto-organizada. Los usuarios pueden utilizar Looker a través del navegador para explorar datos, crear visualizaciones y paneles, programar informes y compartir sus conocimientos con colegas. Looker proporciona un amplio conjunto de herramientas para incrustar estas características en otras aplicaciones y una API -para integrar datos con otras aplicaciones. - -Función: - -- Desarrollo fácil y ágil utilizando LookML, un lenguaje que soporta curado - [Modelado de datos](https://looker.com/platform/data-modeling) para apoyar a los redactores de informes y a los usuarios finales. -- Potente integración de flujo de trabajo a través de Looker's [Acciones de datos](https://looker.com/platform/actions). - -[Cómo configurar ClickHouse en Looker.](https://docs.looker.com/setup-and-management/database-config/clickhouse) - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/third-party/gui/) diff --git a/docs/es/interfaces/third-party/index.md b/docs/es/interfaces/third-party/index.md deleted file mode 100644 index adf50b05cdf..00000000000 --- a/docs/es/interfaces/third-party/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: tercero -toc_priority: 24 ---- - - diff --git a/docs/es/interfaces/third-party/integrations.md b/docs/es/interfaces/third-party/integrations.md deleted file mode 100644 index 7588bef0230..00000000000 --- a/docs/es/interfaces/third-party/integrations.md +++ /dev/null @@ -1,108 +0,0 @@ ---- -toc_priority: 27 -toc_title: Integrations ---- - -# Integration Libraries from Third-party Developers {#integration-libraries-from-third-party-developers} - -!!! warning "Disclaimer" - Yandex does **not** maintain the tools and libraries listed below and haven’t done any extensive testing to ensure their quality. - -## Infrastructure Products {#infrastructure-products} - -- Relational database management systems - - [MySQL](https://www.mysql.com) - - [mysql2ch](https://github.com/long2ice/mysql2ch) - - [ProxySQL](https://github.com/sysown/proxysql/wiki/ClickHouse-Support) - - [clickhouse-mysql-data-reader](https://github.com/Altinity/clickhouse-mysql-data-reader) - - [horgh-replicator](https://github.com/larsnovikov/horgh-replicator) - - [PostgreSQL](https://www.postgresql.org) - - [clickhousedb_fdw](https://github.com/Percona-Lab/clickhousedb_fdw) - - [infi.clickhouse_fdw](https://github.com/Infinidat/infi.clickhouse_fdw) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pg2ch](https://github.com/mkabilov/pg2ch) - - [clickhouse_fdw](https://github.com/adjust/clickhouse_fdw) - - [MSSQL](https://en.wikipedia.org/wiki/Microsoft_SQL_Server) - - [ClickHouseMigrator](https://github.com/zlzforever/ClickHouseMigrator) -- Message queues - - [Kafka](https://kafka.apache.org) - - [clickhouse_sinker](https://github.com/housepower/clickhouse_sinker) (uses [Go client](https://github.com/ClickHouse/clickhouse-go/)) - - [stream-loader-clickhouse](https://github.com/adform/stream-loader) -- Stream processing - - [Flink](https://flink.apache.org) - - [flink-clickhouse-sink](https://github.com/ivi-ru/flink-clickhouse-sink) -- Object storages - - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) -- Container orchestration - - [Kubernetes](https://kubernetes.io) - - [clickhouse-operator](https://github.com/Altinity/clickhouse-operator) -- Configuration management - - [puppet](https://puppet.com) - - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - - [mfedotov/clickhouse](https://forge.puppet.com/mfedotov/clickhouse) -- Monitoring - - [Graphite](https://graphiteapp.org) - - [graphouse](https://github.com/yandex/graphouse) - - [carbon-clickhouse](https://github.com/lomik/carbon-clickhouse) + - - [graphite-clickhouse](https://github.com/lomik/graphite-clickhouse) - - [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer) - optimizes staled partitions in [\*GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md#graphitemergetree) if rules from [rollup configuration](../../engines/table-engines/mergetree-family/graphitemergetree.md#rollup-configuration) could be applied - - [Grafana](https://grafana.com/) - - [clickhouse-grafana](https://github.com/Vertamedia/clickhouse-grafana) - - [Prometheus](https://prometheus.io/) - - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - - [PromHouse](https://github.com/Percona-Lab/PromHouse) - - [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/)) - - [Nagios](https://www.nagios.org/) - - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) - - [check_clickhouse.py](https://github.com/innogames/igmonplugins/blob/master/src/check_clickhouse.py) - - [Zabbix](https://www.zabbix.com) - - [clickhouse-zabbix-template](https://github.com/Altinity/clickhouse-zabbix-template) - - [Sematext](https://sematext.com/) - - [clickhouse integration](https://github.com/sematext/sematext-agent-integrations/tree/master/clickhouse) -- Logging - - [rsyslog](https://www.rsyslog.com/) - - [omclickhouse](https://www.rsyslog.com/doc/master/configuration/modules/omclickhouse.html) - - [fluentd](https://www.fluentd.org) - - [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io)) - - [logagent](https://www.sematext.com/logagent) - - [logagent output-plugin-clickhouse](https://sematext.com/docs/logagent/output-plugin-clickhouse/) -- Geo - - [MaxMind](https://dev.maxmind.com/geoip/) - - [clickhouse-maxmind-geoip](https://github.com/AlexeyKupershtokh/clickhouse-maxmind-geoip) - -## Programming Language Ecosystems {#programming-language-ecosystems} - -- Python - - [SQLAlchemy](https://www.sqlalchemy.org) - - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pandas](https://pandas.pydata.org) - - [pandahouse](https://github.com/kszucs/pandahouse) -- PHP - - [Doctrine](https://www.doctrine-project.org/) - - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) -- R - - [dplyr](https://db.rstudio.com/dplyr/) - - [RClickHouse](https://github.com/IMSMWU/RClickHouse) (uses [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) -- Java - - [Hadoop](http://hadoop.apache.org) - - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../../sql-reference/table-functions/jdbc.md)) -- Scala - - [Akka](https://akka.io) - - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) -- C# - - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) - - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) -- Elixir - - [Ecto](https://github.com/elixir-ecto/ecto) - - [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto) -- Ruby - - [Ruby on Rails](https://rubyonrails.org/) - - [activecube](https://github.com/bitquery/activecube) - - [ActiveRecord](https://github.com/PNixx/clickhouse-activerecord) - - [GraphQL](https://github.com/graphql) - - [activecube-graphql](https://github.com/bitquery/activecube-graphql) - -[Original article](https://clickhouse.tech/docs/en/interfaces/third-party/integrations/) diff --git a/docs/es/interfaces/third-party/proxy.md b/docs/es/interfaces/third-party/proxy.md deleted file mode 100644 index e1aabf8fce4..00000000000 --- a/docs/es/interfaces/third-party/proxy.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 29 -toc_title: Proxy ---- - -# Servidores proxy de desarrolladores de terceros {#proxy-servers-from-third-party-developers} - -## chproxy {#chproxy} - -[chproxy](https://github.com/Vertamedia/chproxy), es un proxy HTTP y equilibrador de carga para la base de datos ClickHouse. - -Función: - -- Enrutamiento por usuario y almacenamiento en caché de respuestas. -- Flexible límites. -- Renovación automática del certificado SSL. - -Implementado en Go. - -## Bienvenido a WordPress {#kittenhouse} - -[Bienvenido a WordPress.](https://github.com/VKCOM/kittenhouse) está diseñado para ser un proxy local entre ClickHouse y el servidor de aplicaciones en caso de que sea imposible o inconveniente almacenar los datos INSERT en el lado de su aplicación. - -Función: - -- Almacenamiento en búfer de datos en memoria y en disco. -- Enrutamiento por tabla. -- Equilibrio de carga y comprobación de estado. - -Implementado en Go. - -## Bienvenidos al Portal de Licitación Electrónica de Licitación Electrónica {#clickhouse-bulk} - -[Bienvenidos al Portal de Licitación Electrónica de Licitación Electrónica](https://github.com/nikepan/clickhouse-bulk) es un simple colector de insertos ClickHouse. - -Función: - -- Agrupe las solicitudes y envíe por umbral o intervalo. -- Múltiples servidores remotos. -- Autenticación básica. - -Implementado en Go. - -[Artículo Original](https://clickhouse.tech/docs/en/interfaces/third-party/proxy/) diff --git a/docs/es/introduction/adopters.md b/docs/es/introduction/adopters.md deleted file mode 100644 index 4c0aa78d57b..00000000000 --- a/docs/es/introduction/adopters.md +++ /dev/null @@ -1,86 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 8 -toc_title: Adoptante ---- - -# Adoptadores de ClickHouse {#clickhouse-adopters} - -!!! warning "Descargo" - La siguiente lista de empresas que utilizan ClickHouse y sus historias de éxito se recopila a partir de fuentes públicas, por lo que podría diferir de la realidad actual. Le agradeceríamos que compartiera la historia de adoptar ClickHouse en su empresa y [agregarlo a la lista](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md), pero por favor asegúrese de que usted no tendrá ningunos problemas de NDA haciendo así. Proporcionar actualizaciones con publicaciones de otras compañías también es útil. - -| Empresa | Industria | Usecase | Tamaño de clúster | (Un)Tamaño de datos comprimidos\* | Referencia | -|-------------------------------------------------------------------------------------------------|------------------------------------|-----------------------------|------------------------------------------------------------------|-------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| 2gis | Asignar | Monitoreo | — | — | [Charla en ruso, julio 2019](https://youtu.be/58sPkXfq6nw) | -| Aloha Browser | Aplicación móvil | Backend del navegador | — | — | [Diapositivas en ruso, mayo 2019](https://github.com/yandex/clickhouse-presentations/blob/master/meetup22/aloha.pdf) | -| Amadeus | Viaje | Analítica | — | — | [Comunicado de prensa, abril de 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | -| Appsflyer | Análisis móvil | Producto principal | — | — | [Charla en ruso, julio 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | -| ArenaData | Plataforma de datos | Producto principal | — | — | [Diapositivas en ruso, diciembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | -| Badoo | Citas | Serie de tiempo | — | — | [Diapositivas en ruso, diciembre 2019](https://presentations.clickhouse.tech/meetup38/forecast.pdf) | -| Benocs | Telemetría y análisis de red | Producto principal | — | — | [Diapositivas en español, octubre de 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | -| Bloomberg | Finanzas, Medios | Monitoreo | 102 servidores | — | [Diapositivas, Mayo 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | -| Bloxy | Blockchain | Analítica | — | — | [Diapositivas en ruso, agosto 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | -| Dataliance para China Telecom | Telecomunicaciones | Analítica | — | — | [Diapositivas en chino, enero 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | -| CARTO | Inteligencia de negocios | Análisis geográfico | — | — | [Procesamiento geoespacial con ClickHouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) | -| CERN | Investigación | Experimento | — | — | [Comunicado de prensa, abril de 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) | -| Cisco | Red | Análisis de tráfico | — | — | [Charla relámpago, octubre 2019](https://youtu.be/-hI1vDR2oPY?t=5057) | -| Citadel Securities | Financiación | — | — | — | [Contribución, marzo 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | -| Más información | Taxi | Analítica | — | — | [Blog Post en ruso, marzo 2020](https://habr.com/en/company/citymobil/blog/490660/) | -| ContentSquare | Análisis web | Producto principal | — | — | [Publicación de blog en francés, noviembre 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) | -| Cloudflare | CDN | Análisis de tráfico | 36 servidores | — | [Mensaje del blog, Mayo 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Mensaje del blog, marzo 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) | -| Corunet | Analítica | Producto principal | — | — | [Diapositivas en español, Abril 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) | -| CraiditX 氪信 | Finanzas AI | Análisis | — | — | [Diapositivas en español, noviembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | -| Criteo | Menor | Producto principal | — | — | [Diapositivas en español, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) | -| Deutsche Bank | Financiación | BI Analytics | — | — | [Diapositivas en español, octubre 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | -| Diva-e | Consultoría digital | Producto principal | — | — | [Diapositivas en español, septiembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | -| Exness | Comercio | Métricas, Registro | — | — | [Charla en ruso, mayo 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | -| Sistema abierto. | Red Ad | Producto principal | — | — | [Publicación de blog en japonés, julio 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | -| HUYA | Video Streaming | Analítica | — | — | [Diapositivas en chino, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | -| Idealista | Inmobiliario | Analítica | — | — | [Blog Post en Inglés, Abril 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | -| Infovista | Red | Analítica | — | — | [Diapositivas en español, octubre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) | -| InnoGames | Juego | Métricas, Registro | — | — | [Diapositivas en ruso, septiembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) | -| Integros | Plataforma para servicios de video | Analítica | — | — | [Diapositivas en ruso, mayo 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | -| Datos de Kodiak | Nube | Producto principal | — | — | [Diapositivas en Engish, Abril 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | -| Kontur | Desarrollo de software | Métricas | — | — | [Charla en ruso, noviembre 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | -| Sistema abierto. | Red Ad | Producto principal | 75 servidores (3 réplicas) | 5.27 PiB | [Publicación de blog en ruso, febrero 2017](https://habr.com/en/post/322620/) | -| Soluciones en la nube de Mail.ru | Servicios en la nube | Producto principal | — | — | [Artículo en ruso](https://mcs.mail.ru/help/db-create/clickhouse#) | -| Mensaje de pájaro | Telecomunicaciones | Estadísticas | — | — | [Diapositivas en español, noviembre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | -| MGID | Red Ad | Analítica Web | — | — | [Publicación de blog en ruso, abril 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | -| UnoAPM | Supervisión y análisis de datos | Producto principal | — | — | [Diapositivas en chino, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | -| Pragma Innovation | Telemetría y Análisis de Big Data | Producto principal | — | — | [Diapositivas en español, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) | -| QINGCLOUD | Servicios en la nube | Producto principal | — | — | [Diapositivas en chino, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) | -| Qrator | Protección DDoS | Producto principal | — | — | [Blog Post, marzo 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) | -| Percent 百分点 | Analítica | Producto principal | — | — | [Diapositivas en chino, junio 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | -| Rambler | Servicios de Internet | Analítica | — | — | [Charla en ruso, abril 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | -| Tencent | Mensajería | Tala | — | — | [Charla en chino, noviembre 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | -| Traffic Stars | Red AD | — | — | — | [Diapositivas en ruso, mayo 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | -| S7 Airlines | Aérea | Métricas, Registro | — | — | [Charla en ruso, marzo 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | -| SEMrush | Marketing | Producto principal | — | — | [Diapositivas en ruso, agosto 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | -| scireum GmbH | Comercio electrónico | Producto principal | — | — | [Charla en alemán, febrero de 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | -| Centinela | Desarrollador de software | Backend para el producto | — | — | [Publicación de blog en inglés, mayo 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | -| SGK | Gobierno Seguridad Social | Analítica | — | — | [Diapositivas en español, noviembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | -| el seo.¿ | Analítica | Producto principal | — | — | [Diapositivas en español, noviembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | -| Sina | Noticia | — | — | — | [Diapositivas en chino, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | -| SMI2 | Noticia | Analítica | — | — | [Blog Post en ruso, noviembre 2017](https://habr.com/ru/company/smi2/blog/314558/) | -| Salto | Análisis de negocios | Producto principal | — | — | [Diapositivas en español, enero 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | -| Spotify | Sica | Experimentación | — | — | [Diapositivas, julio 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | -| Tencent | Grandes Datos | Procesamiento de datos | — | — | [Diapositivas en chino, octubre 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | -| Más información | Taxi | Tala | — | — | [Diapositivas, febrero de 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | -| VKontakte | Red social | Estadísticas, Registro | — | — | [Diapositivas en ruso, agosto 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | -| Método de codificación de datos: | Soluciones de TI | Analítica | — | — | [Diapositivas en ruso, mayo 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | -| Xiaoxin Tech | Educación | Propósito común | — | — | [Diapositivas en español, noviembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) | -| Ximalaya | Compartir audio | OLAP | — | — | [Diapositivas en español, noviembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) | -| Yandex Cloud | Nube pública | Producto principal | — | — | [Charla en ruso, diciembre 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | -| Yandex DataLens | Inteligencia de negocios | Producto principal | — | — | [Diapositivas en ruso, diciembre 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) | -| Yandex Market | Comercio electrónico | Métricas, Registro | — | — | [Charla en ruso, enero 2019](https://youtu.be/_l1qP0DyBcA?t=478) | -| Yandex Metrica | Análisis web | Producto principal | 360 servidores en un clúster, 1862 servidores en un departamento | 66.41 PiB / 5.68 PiB | [Diapositivas, febrero de 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) | -| ЦВТ | Desarrollo de software | Métricas, Registro | — | — | [Blog Post, marzo 2019, en ruso](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | -| МКБ | Banco | Supervisión del sistema web | — | — | [Diapositivas en ruso, septiembre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | -| Jinshuju 金数据 | BI Analytics | Producto principal | — | — | [Diapositivas en chino, octubre 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | -| Instana | Plataforma APM | Producto principal | — | — | [Publicación de Twitter](https://twitter.com/mieldonkers/status/1248884119158882304) | -| Wargaming | Juego | | — | — | [Entrevista](https://habr.com/en/post/496954/) | -| Crazypanda | Juego | | — | — | Sesión en vivo en ClickHouse meetup | -| FunCorp | Juego | | — | — | [Artículo](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | - -[Artículo Original](https://clickhouse.tech/docs/en/introduction/adopters/) diff --git a/docs/es/introduction/distinctive-features.md b/docs/es/introduction/distinctive-features.md deleted file mode 100644 index 154b12a65e9..00000000000 --- a/docs/es/introduction/distinctive-features.md +++ /dev/null @@ -1,77 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 4 -toc_title: "Caracter\xEDsticas distintivas" ---- - -# Características distintivas de ClickHouse {#distinctive-features-of-clickhouse} - -## DBMS orientado a columnas verdaderas {#true-column-oriented-dbms} - -En un verdadero DBMS orientado a columnas, no se almacenan datos adicionales con los valores. Entre otras cosas, esto significa que los valores de longitud constante deben ser compatibles, para evitar almacenar su longitud “number” al lado de los valores. Como ejemplo, mil millones de valores de tipo UInt8 deberían consumir alrededor de 1 GB sin comprimir, o esto afecta fuertemente el uso de la CPU. Es esencial almacenar los datos de forma compacta (sin “garbage”) incluso sin comprimir, ya que la velocidad de descompresión (uso de CPU) depende principalmente del volumen de datos sin comprimir. - -Vale la pena señalar porque hay sistemas que pueden almacenar valores de diferentes columnas por separado, pero que no pueden procesar efectivamente las consultas analíticas debido a su optimización para otros escenarios. Los ejemplos son HBase, BigTable, Cassandra e HyperTable. En estos sistemas, obtendría un rendimiento de alrededor de cien mil filas por segundo, pero no cientos de millones de filas por segundo. - -También vale la pena señalar que ClickHouse es un sistema de administración de bases de datos, no una sola base de datos. ClickHouse permite crear tablas y bases de datos en tiempo de ejecución, cargar datos y ejecutar consultas sin volver a configurar y reiniciar el servidor. - -## Compresión de datos {#data-compression} - -Algunos DBMS orientados a columnas (InfiniDB CE y MonetDB) no utilizan la compresión de datos. Sin embargo, la compresión de datos juega un papel clave para lograr un rendimiento excelente. - -## Almacenamiento en disco de datos {#disk-storage-of-data} - -Mantener los datos físicamente ordenados por clave principal permite extraer datos para sus valores específicos o rangos de valores con baja latencia, menos de unas pocas docenas de milisegundos. Algunos DBMS orientados a columnas (como SAP HANA y Google PowerDrill) solo pueden funcionar en RAM. Este enfoque fomenta la asignación de un presupuesto de hardware más grande que el necesario para el análisis en tiempo real. ClickHouse está diseñado para funcionar en discos duros normales, lo que significa que el costo por GB de almacenamiento de datos es bajo, pero SSD y RAM adicional también se utilizan completamente si están disponibles. - -## Procesamiento paralelo en varios núcleos {#parallel-processing-on-multiple-cores} - -Las consultas grandes se paralelizan naturalmente, tomando todos los recursos necesarios disponibles en el servidor actual. - -## Procesamiento distribuido en varios servidores {#distributed-processing-on-multiple-servers} - -Casi ninguno de los DBMS columnar mencionados anteriormente tiene soporte para el procesamiento de consultas distribuidas. -En ClickHouse, los datos pueden residir en diferentes fragmentos. Cada fragmento puede ser un grupo de réplicas utilizadas para la tolerancia a errores. Todos los fragmentos se utilizan para ejecutar una consulta en paralelo, de forma transparente para el usuario. - -## Soporte SQL {#sql-support} - -ClickHouse admite un lenguaje de consulta declarativo basado en SQL que es idéntico al estándar SQL en muchos casos. -Las consultas admitidas incluyen GROUP BY, ORDER BY, subconsultas en cláusulas FROM, IN y JOIN y subconsultas escalares. -No se admiten subconsultas y funciones de ventana dependientes. - -## Motor del vector {#vector-engine} - -Los datos no solo se almacenan mediante columnas, sino que se procesan mediante vectores (partes de columnas), lo que permite lograr una alta eficiencia de CPU. - -## Actualizaciones de datos en tiempo real {#real-time-data-updates} - -ClickHouse admite tablas con una clave principal. Para realizar consultas rápidamente en el rango de la clave principal, los datos se ordenan de forma incremental utilizando el árbol de combinación. Debido a esto, los datos se pueden agregar continuamente a la tabla. No se toman bloqueos cuando se ingieren nuevos datos. - -## Indice {#index} - -Tener un dato ordenado físicamente por clave principal permite extraer datos para sus valores específicos o rangos de valores con baja latencia, menos de unas pocas docenas de milisegundos. - -## Adecuado para consultas en línea {#suitable-for-online-queries} - -La baja latencia significa que las consultas se pueden procesar sin demora y sin intentar preparar una respuesta por adelantado, justo en el mismo momento mientras se carga la página de la interfaz de usuario. En otras palabras, en línea. - -## Soporte para cálculos aproximados {#support-for-approximated-calculations} - -ClickHouse proporciona varias formas de intercambiar precisión por rendimiento: - -1. Funciones agregadas para el cálculo aproximado del número de valores distintos, medianas y cuantiles. -2. Ejecutar una consulta basada en una parte (muestra) de datos y obtener un resultado aproximado. En este caso, se recuperan proporcionalmente menos datos del disco. -3. Ejecutar una agregación para un número limitado de claves aleatorias, en lugar de para todas las claves. Bajo ciertas condiciones para la distribución de claves en los datos, esto proporciona un resultado razonablemente preciso mientras se utilizan menos recursos. - -## Replicación de datos e integridad de datos {#data-replication-and-data-integrity-support} - -ClickHouse utiliza la replicación multi-maestro asincrónica. Después de escribir en cualquier réplica disponible, todas las réplicas restantes recuperan su copia en segundo plano. El sistema mantiene datos idénticos en diferentes réplicas. La recuperación después de la mayoría de las fallas se realiza automáticamente, o semiautomáticamente en casos complejos. - -Para obtener más información, consulte la sección [Replicación de datos](../engines/table-engines/mergetree-family/replication.md). - -## Características que pueden considerarse desventajas {#clickhouse-features-that-can-be-considered-disadvantages} - -1. No hay transacciones completas. -2. Falta de capacidad para modificar o eliminar datos ya insertados con alta tasa y baja latencia. Hay eliminaciones y actualizaciones por lotes disponibles para limpiar o modificar datos, por ejemplo, para cumplir con [GDPR](https://gdpr-info.eu). -3. El índice disperso hace que ClickHouse no sea tan adecuado para consultas de puntos que recuperan filas individuales por sus claves. - -[Artículo Original](https://clickhouse.tech/docs/en/introduction/distinctive_features/) diff --git a/docs/es/introduction/history.md b/docs/es/introduction/history.md deleted file mode 100644 index 7311fa01959..00000000000 --- a/docs/es/introduction/history.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 7 -toc_title: Historia ---- - -# Historial de ClickHouse {#clickhouse-history} - -ClickHouse se ha desarrollado inicialmente para alimentar [El Yandex.Métrica](https://metrica.yandex.com/), [la segunda plataforma de análisis web más grande del mundo](http://w3techs.com/technologies/overview/traffic_analysis/all), y sigue siendo el componente central de este sistema. Con más de 13 billones de registros en la base de datos y más de 20 mil millones de eventos diarios, ClickHouse permite generar informes personalizados sobre la marcha directamente a partir de datos no agregados. Este artículo cubre brevemente los objetivos de ClickHouse en las primeras etapas de su desarrollo. - -El Yandex.Metrica construye informes personalizados sobre la marcha basados en hits y sesiones, con segmentos arbitrarios definidos por el usuario. Hacerlo a menudo requiere construir agregados complejos, como el número de usuarios únicos. Los nuevos datos para crear un informe llegan en tiempo real. - -A partir de abril de 2014, Yandex.Metrica estaba rastreando alrededor de 12 mil millones de eventos (vistas de páginas y clics) diariamente. Todos estos eventos deben almacenarse para crear informes personalizados. Una sola consulta puede requerir escanear millones de filas en unos pocos cientos de milisegundos, o cientos de millones de filas en solo unos segundos. - -## Uso en Yandex.Metrica y otros servicios de Yandex {#usage-in-yandex-metrica-and-other-yandex-services} - -ClickHouse sirve para múltiples propósitos en Yandex.Métrica. -Su tarea principal es crear informes en modo en línea utilizando datos no agregados. Utiliza un clúster de 374 servidores, que almacenan más de 20,3 billones de filas en la base de datos. El volumen de datos comprimidos es de aproximadamente 2 PB, sin tener en cuenta duplicados y réplicas. El volumen de datos sin comprimir (en formato TSV) sería de aproximadamente 17 PB. - -ClickHouse también juega un papel clave en los siguientes procesos: - -- Almacenamiento de datos para Session Replay de Yandex.Métrica. -- Procesamiento de datos intermedios. -- Creación de informes globales con Analytics. -- Ejecutar consultas para depurar el Yandex.Motor Metrica. -- Análisis de registros desde la API y la interfaz de usuario. - -Hoy en día, hay varias docenas de instalaciones de ClickHouse en otros servicios y departamentos de Yandex: verticales de búsqueda, comercio electrónico, publicidad, análisis de negocios, desarrollo móvil, servicios personales y otros. - -## Datos agregados y no agregados {#aggregated-and-non-aggregated-data} - -Existe una opinión generalizada de que para calcular las estadísticas de manera efectiva, debe agregar datos ya que esto reduce el volumen de datos. - -Pero la agregación de datos viene con muchas limitaciones: - -- Debe tener una lista predefinida de los informes necesarios. -- El usuario no puede hacer informes personalizados. -- Al agregar sobre un gran número de claves distintas, el volumen de datos apenas se reduce, por lo que la agregación es inútil. -- Para un gran número de informes, hay demasiadas variaciones de agregación (explosión combinatoria). -- Al agregar claves con alta cardinalidad (como las URL), el volumen de datos no se reduce en mucho (menos del doble). -- Por esta razón, el volumen de datos con agregación podría crecer en lugar de reducirse. -- Los usuarios no ven todos los informes que generamos para ellos. Una gran parte de esos cálculos es inútil. -- La integridad lógica de los datos puede ser violada para varias agregaciones. - -Si no agregamos nada y trabajamos con datos no agregados, esto podría reducir el volumen de cálculos. - -Sin embargo, con la agregación, una parte significativa del trabajo se desconecta y se completa con relativa calma. Por el contrario, los cálculos en línea requieren calcular lo más rápido posible, ya que el usuario está esperando el resultado. - -El Yandex.Metrica tiene un sistema especializado para agregar datos llamado Metrage, que se utilizó para la mayoría de los informes. -A partir de 2009, Yandex.Metrica también utilizó una base de datos OLAP especializada para datos no agregados llamada OLAPServer, que anteriormente se usaba para el generador de informes. -OLAPServer funcionó bien para datos no agregados, pero tenía muchas restricciones que no permitían que se utilizara para todos los informes según lo deseado. Estos incluyeron la falta de soporte para tipos de datos (solo números) y la incapacidad de actualizar datos de forma incremental en tiempo real (solo se podía hacer reescribiendo datos diariamente). OLAPServer no es un DBMS, sino una base de datos especializada. - -El objetivo inicial de ClickHouse era eliminar las limitaciones de OLAPServer y resolver el problema de trabajar con datos no agregados para todos los informes, pero a lo largo de los años, se ha convertido en un sistema de gestión de bases de datos de propósito general adecuado para una amplia gama de tareas analíticas. - -[Artículo Original](https://clickhouse.tech/docs/en/introduction/history/) diff --git a/docs/es/introduction/index.md b/docs/es/introduction/index.md deleted file mode 100644 index 7026dc800e4..00000000000 --- a/docs/es/introduction/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Implantaci\xF3n" -toc_priority: 1 ---- - - diff --git a/docs/es/introduction/performance.md b/docs/es/introduction/performance.md deleted file mode 100644 index 01640439128..00000000000 --- a/docs/es/introduction/performance.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 6 -toc_title: Rendimiento ---- - -# Rendimiento {#performance} - -De acuerdo con los resultados de las pruebas internas en Yandex, ClickHouse muestra el mejor rendimiento (tanto el mayor rendimiento para consultas largas como la menor latencia en consultas cortas) para escenarios operativos comparables entre los sistemas de su clase que estaban disponibles para pruebas. Puede ver los resultados de la prueba en un [página separada](https://clickhouse.tech/benchmark/dbms/). - -Numerosos puntos de referencia independientes llegaron a conclusiones similares. No son difíciles de encontrar mediante una búsqueda en Internet, o se puede ver [nuestra pequeña colección de enlaces relacionados](https://clickhouse.tech/#independent-benchmarks). - -## Rendimiento para una única consulta grande {#throughput-for-a-single-large-query} - -El rendimiento se puede medir en filas por segundo o megabytes por segundo. Si los datos se colocan en la caché de la página, una consulta que no es demasiado compleja se procesa en hardware moderno a una velocidad de aproximadamente 2-10 GB / s de datos sin comprimir en un solo servidor (para los casos más sencillos, la velocidad puede alcanzar 30 GB / s). Si los datos no se colocan en la memoria caché de la página, la velocidad depende del subsistema de disco y la velocidad de compresión de datos. Por ejemplo, si el subsistema de disco permite leer datos a 400 MB/s y la tasa de compresión de datos es 3, se espera que la velocidad sea de alrededor de 1,2 GB/s. Para obtener la velocidad en filas por segundo, divida la velocidad en bytes por segundo por el tamaño total de las columnas utilizadas en la consulta. Por ejemplo, si se extraen 10 bytes de columnas, se espera que la velocidad sea de alrededor de 100-200 millones de filas por segundo. - -La velocidad de procesamiento aumenta casi linealmente para el procesamiento distribuido, pero solo si el número de filas resultantes de la agregación o la clasificación no es demasiado grande. - -## Latencia al procesar consultas cortas {#latency-when-processing-short-queries} - -Si una consulta usa una clave principal y no selecciona demasiadas columnas y filas para procesar (cientos de miles), puede esperar menos de 50 milisegundos de latencia (dígitos individuales de milisegundos en el mejor de los casos) si los datos se colocan en la memoria caché de la página. De lo contrario, la latencia está dominada principalmente por el número de búsquedas. Si utiliza unidades de disco giratorias, para un sistema que no está sobrecargado, la latencia se puede estimar con esta fórmula: `seek time (10 ms) * count of columns queried * count of data parts`. - -## Rendimiento al procesar una gran cantidad de consultas cortas {#throughput-when-processing-a-large-quantity-of-short-queries} - -En las mismas condiciones, ClickHouse puede manejar varios cientos de consultas por segundo en un solo servidor (hasta varios miles en el mejor de los casos). Dado que este escenario no es típico para DBMS analíticos, se recomienda esperar un máximo de 100 consultas por segundo. - -## Rendimiento al insertar datos {#performance-when-inserting-data} - -Recomendamos insertar datos en paquetes de al menos 1000 filas o no más de una sola solicitud por segundo. Al insertar en una tabla MergeTree desde un volcado separado por tabuladores, la velocidad de inserción puede ser de 50 a 200 MB/s. Si las filas insertadas tienen alrededor de 1 Kb de tamaño, la velocidad será de 50,000 a 200,000 filas por segundo. Si las filas son pequeñas, el rendimiento puede ser mayor en filas por segundo (en los datos del sistema Banner -`>` 500.000 filas por segundo; en datos de grafito -`>` 1.000.000 de filas por segundo). Para mejorar el rendimiento, puede realizar varias consultas INSERT en paralelo, que se escala linealmente. - -[Artículo Original](https://clickhouse.tech/docs/en/introduction/performance/) diff --git a/docs/es/operations/access-rights.md b/docs/es/operations/access-rights.md deleted file mode 100644 index 6c777d9f081..00000000000 --- a/docs/es/operations/access-rights.md +++ /dev/null @@ -1,143 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 48 -toc_title: "Control de acceso y gesti\xF3n de cuentas" ---- - -# Control de acceso y gestión de cuentas {#access-control} - -ClickHouse admite la administración de control de acceso basada en [RBAC](https://en.wikipedia.org/wiki/Role-based_access_control) enfoque. - -Entidades de acceso de ClickHouse: -- [Cuenta de usuario](#user-account-management) -- [Rol](#role-management) -- [Política de fila](#row-policy-management) -- [Perfil de configuración](#settings-profiles-management) -- [Cuota](#quotas-management) - -Puede configurar entidades de acceso utilizando: - -- Flujo de trabajo controlado por SQL. - - Es necesario [permitir](#enabling-access-control) esta funcionalidad. - -- Servidor [archivos de configuración](configuration-files.md) `users.xml` y `config.xml`. - -Se recomienda utilizar el flujo de trabajo controlado por SQL. Ambos métodos de configuración funcionan simultáneamente, por lo que si utiliza los archivos de configuración del servidor para administrar cuentas y derechos de acceso, puede pasar suavemente al flujo de trabajo controlado por SQL. - -!!! note "Advertencia" - No puede administrar la misma entidad de acceso mediante ambos métodos de configuración simultáneamente. - -## Uso {#access-control-usage} - -De forma predeterminada, el servidor ClickHouse proporciona la cuenta de usuario `default` que no está permitido usar control de acceso controlado por SQL y administración de cuentas, pero tiene todos los derechos y permisos. El `default` cuenta de usuario se utiliza en cualquier caso cuando el nombre de usuario no está definido, por ejemplo, al iniciar sesión desde el cliente o en consultas distribuidas. En el procesamiento de consultas distribuidas se utiliza una cuenta de usuario predeterminada, si la configuración del servidor o clúster no [usuario y contraseña](../engines/table-engines/special/distributed.md) propiedad. - -Si acaba de comenzar a usar ClickHouse, puede usar el siguiente escenario: - -1. [Permitir](#enabling-access-control) Control de acceso basado en SQL y gestión de cuentas `default` usuario. -2. Inicie sesión bajo el `default` cuenta de usuario y crear todos los usuarios. No olvides crear una cuenta de administrador (`GRANT ALL ON *.* WITH GRANT OPTION TO admin_user_account`). -3. [Restringir permisos](settings/permissions-for-queries.md#permissions_for_queries) para el `default` usuario y deshabilitar el control de acceso impulsado por SQL y la administración de cuentas para ello. - -### Propiedades de la solución actual {#access-control-properties} - -- Puede conceder permisos para bases de datos y tablas incluso si no existen. -- Si se eliminó una tabla, no se revocarán todos los privilegios que corresponden a esta tabla. Por lo tanto, si se crea una nueva tabla más tarde con el mismo nombre, todos los privilegios vuelven a ser reales. Para revocar los privilegios correspondientes a la tabla eliminada, debe realizar, por ejemplo, el `REVOKE ALL PRIVILEGES ON db.table FROM ALL` consulta. -- No hay ninguna configuración de por vida para los privilegios. - -## Cuenta de usuario {#user-account-management} - -Una cuenta de usuario es una entidad de acceso que permite autorizar a alguien en ClickHouse. Una cuenta de usuario contiene: - -- Información de identificación. -- [Privilegio](../sql-reference/statements/grant.md#grant-privileges) que definen un ámbito de consultas que el usuario puede realizar. -- Hosts desde los que se permite la conexión al servidor ClickHouse. -- Roles otorgados y predeterminados. -- Configuración con sus restricciones que se aplican de forma predeterminada en el inicio de sesión del usuario. -- Perfiles de configuración asignados. - -Los privilegios a una cuenta de usuario pueden ser otorgados por el [GRANT](../sql-reference/statements/grant.md) consulta o asignando [rol](#role-management). Para revocar privilegios de un usuario, ClickHouse proporciona el [REVOKE](../sql-reference/statements/revoke.md) consulta. Para listar los privilegios de un usuario, utilice - [SHOW GRANTS](../sql-reference/statements/show.md#show-grants-statement) instrucción. - -Consultas de gestión: - -- [CREATE USER](../sql-reference/statements/create.md#create-user-statement) -- [ALTER USER](../sql-reference/statements/alter.md#alter-user-statement) -- [DROP USER](../sql-reference/statements/misc.md#drop-user-statement) -- [SHOW CREATE USER](../sql-reference/statements/show.md#show-create-user-statement) - -### Ajustes Aplicación {#access-control-settings-applying} - -Los ajustes se pueden establecer de diferentes maneras: para una cuenta de usuario, en sus roles y perfiles de configuración concedidos. En un inicio de sesión de usuario, si se establece una configuración en diferentes entidades de acceso, el valor y las restricciones de esta configuración se aplican mediante las siguientes prioridades (de mayor a menor): - -1. Configuración de la cuenta de usuario. -2. La configuración de los roles predeterminados de la cuenta de usuario. Si se establece una configuración en algunos roles, el orden de la configuración que se aplica no está definido. -3. La configuración de los perfiles de configuración asignados a un usuario o a sus roles predeterminados. Si se establece una configuración en algunos perfiles, el orden de aplicación de la configuración no está definido. -4. Ajustes aplicados a todo el servidor de forma predeterminada o desde el [perfil predeterminado](server-configuration-parameters/settings.md#default-profile). - -## Rol {#role-management} - -Role es un contenedor para las entidades de acceso que se pueden conceder a una cuenta de usuario. - -El rol contiene: - -- [Privilegio](../sql-reference/statements/grant.md#grant-privileges) -- Configuración y restricciones -- Lista de funciones concedidas - -Consultas de gestión: - -- [CREATE ROLE](../sql-reference/statements/create.md#create-role-statement) -- [ALTER ROLE](../sql-reference/statements/alter.md#alter-role-statement) -- [DROP ROLE](../sql-reference/statements/misc.md#drop-role-statement) -- [SET ROLE](../sql-reference/statements/misc.md#set-role-statement) -- [SET DEFAULT ROLE](../sql-reference/statements/misc.md#set-default-role-statement) -- [SHOW CREATE ROLE](../sql-reference/statements/show.md#show-create-role-statement) - -Los privilegios a un rol pueden ser otorgados por el [GRANT](../sql-reference/statements/grant.md) consulta. Para revocar privilegios de un rol, ClickHouse proporciona el [REVOKE](../sql-reference/statements/revoke.md) consulta. - -## Política de fila {#row-policy-management} - -La directiva de filas es un filtro que define qué filas está disponible para un usuario o para un rol. La directiva de filas contiene filtros para una tabla específica y una lista de roles y/o usuarios que deben usar esta directiva de filas. - -Consultas de gestión: - -- [CREATE ROW POLICY](../sql-reference/statements/create.md#create-row-policy-statement) -- [ALTER ROW POLICY](../sql-reference/statements/alter.md#alter-row-policy-statement) -- [DROP ROW POLICY](../sql-reference/statements/misc.md#drop-row-policy-statement) -- [SHOW CREATE ROW POLICY](../sql-reference/statements/show.md#show-create-row-policy-statement) - -## Perfil de configuración {#settings-profiles-management} - -El perfil de configuración es una colección de [configuración](settings/index.md). El perfil de configuración contiene configuraciones y restricciones, y una lista de roles y/o usuarios a los que se aplica esta cuota. - -Consultas de gestión: - -- [CREATE SETTINGS PROFILE](../sql-reference/statements/create.md#create-settings-profile-statement) -- [ALTER SETTINGS PROFILE](../sql-reference/statements/alter.md#alter-settings-profile-statement) -- [DROP SETTINGS PROFILE](../sql-reference/statements/misc.md#drop-settings-profile-statement) -- [SHOW CREATE SETTINGS PROFILE](../sql-reference/statements/show.md#show-create-settings-profile-statement) - -## Cuota {#quotas-management} - -La cuota limita el uso de recursos. Ver [Cuota](quotas.md). - -La cuota contiene un conjunto de límites para algunas duraciones y una lista de roles y / o usuarios que deben usar esta cuota. - -Consultas de gestión: - -- [CREATE QUOTA](../sql-reference/statements/create.md#create-quota-statement) -- [ALTER QUOTA](../sql-reference/statements/alter.md#alter-quota-statement) -- [DROP QUOTA](../sql-reference/statements/misc.md#drop-quota-statement) -- [SHOW CREATE QUOTA](../sql-reference/statements/show.md#show-create-quota-statement) - -## Habilitación del control de acceso basado en SQL y la administración de cuentas {#enabling-access-control} - -- Configure un directorio para el almacenamiento de configuraciones. - - ClickHouse almacena las configuraciones de entidades de acceso en la carpeta [access_control_path](server-configuration-parameters/settings.md#access_control_path) parámetro de configuración del servidor. - -- Habilite el control de acceso controlado por SQL y la administración de cuentas para al menos una cuenta de usuario. - - De forma predeterminada, el control de acceso controlado por SQL y la administración de cuentas se activan para todos los usuarios. Debe configurar al menos un usuario en el `users.xml` archivo de configuración y asigne 1 al [access_management](settings/settings-users.md#access_management-user-setting) configuración. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/access_rights/) diff --git a/docs/es/operations/backup.md b/docs/es/operations/backup.md deleted file mode 100644 index be33851574a..00000000000 --- a/docs/es/operations/backup.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -toc_priority: 49 -toc_title: Copia de seguridad de datos ---- - -# Copia de seguridad de datos {#data-backup} - -Mientras que la [replicación](../engines/table-engines/mergetree-family/replication.md) proporciona protección contra fallos de hardware, no protege de errores humanos: el borrado accidental de datos, elminar la tabla equivocada o una tabla en el clúster equivocado, y bugs de software que dan como resultado un procesado incorrecto de los datos o la corrupción de los datos. En muchos casos, errores como estos afectarán a todas las réplicas. ClickHouse dispone de salvaguardas para prevenir algunos tipos de errores — por ejemplo, por defecto [no se puede simplemente eliminar tablas con un motor similar a MergeTree que contenga más de 50 Gb de datos](server-configuration-parameters/settings.md#max-table-size-to-drop). Sin embargo, estas salvaguardas no cubren todos los casos posibles y pueden eludirse. - -Para mitigar eficazmente los posibles errores humanos, debe preparar cuidadosamente una estrategia para realizar copias de seguridad y restaurar sus datos **previamente**. - -Cada empresa tiene diferentes recursos disponibles y requisitos comerciales, por lo que no existe una solución universal para las copias de seguridad y restauraciones de ClickHouse que se adapten a cada situación. Lo que funciona para un gigabyte de datos probablemente no funcionará para decenas de petabytes. Hay una variedad de posibles enfoques con sus propios pros y contras, que se discutirán a continuación. Es una buena idea utilizar varios enfoques en lugar de uno solo para compensar sus diversas deficiencias. - -!!! note "Nota" - Tenga en cuenta que si realizó una copia de seguridad de algo y nunca intentó restaurarlo, es probable que la restauración no funcione correctamente cuando realmente la necesite (o al menos tomará más tiempo de lo que las empresas pueden tolerar). Por lo tanto, cualquiera que sea el enfoque de copia de seguridad que elija, asegúrese de automatizar el proceso de restauración también y ponerlo en practica en un clúster de ClickHouse de repuesto regularmente. - -## Duplicar datos de origen en otro lugar {#duplicating-source-data-somewhere-else} - -A menudo, los datos que se ingieren en ClickHouse se entregan a través de algún tipo de cola persistente, como [Acerca de nosotros](https://kafka.apache.org). En este caso, es posible configurar un conjunto adicional de suscriptores que leerá el mismo flujo de datos mientras se escribe en ClickHouse y lo almacenará en almacenamiento en frío en algún lugar. La mayoría de las empresas ya tienen algún almacenamiento en frío recomendado por defecto, que podría ser un almacén de objetos o un sistema de archivos distribuido como [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html). - -## Instantáneas del sistema de archivos {#filesystem-snapshots} - -Algunos sistemas de archivos locales proporcionan funcionalidad de instantánea (por ejemplo, [ZFS](https://en.wikipedia.org/wiki/ZFS)), pero podrían no ser la mejor opción para servir consultas en vivo. Una posible solución es crear réplicas adicionales con este tipo de sistema de archivos y excluirlas del [Distribuido](../engines/table-engines/special/distributed.md) tablas que se utilizan para `SELECT` consulta. Las instantáneas en tales réplicas estarán fuera del alcance de cualquier consulta que modifique los datos. Como beneficio adicional, estas réplicas podrían tener configuraciones de hardware especiales con más discos conectados por servidor, lo que sería rentable. - -## Método de codificación de datos: {#clickhouse-copier} - -[Método de codificación de datos:](utilities/clickhouse-copier.md) es una herramienta versátil que se creó inicialmente para volver a dividir tablas de tamaño petabyte. También se puede usar con fines de copia de seguridad y restauración porque copia datos de forma fiable entre tablas y clústeres de ClickHouse. - -Para volúmenes de datos más pequeños, un simple `INSERT INTO ... SELECT ...` a tablas remotas podría funcionar también. - -## Manipulaciones con piezas {#manipulations-with-parts} - -ClickHouse permite usar la consulta `ALTER TABLE ... FREEZE PARTITION ...` para crear una copia local de particiones de tabla. Esto se implementa utilizando enlaces duros a la carpeta `/var/lib/clickhouse/shadow/`, por lo que generalmente no consume espacio adicional en disco para datos antiguos. Las copias creadas de archivos no son manejadas por el servidor ClickHouse, por lo que puede dejarlas allí: tendrá una copia de seguridad simple que no requiere ningún sistema externo adicional, pero seguirá siendo propenso a problemas de hardware. Por esta razón, es mejor copiarlos de forma remota en otra ubicación y luego eliminar las copias locales. Los sistemas de archivos distribuidos y los almacenes de objetos siguen siendo una buena opción para esto, pero los servidores de archivos conectados normales con una capacidad lo suficientemente grande podrían funcionar también (en este caso, la transferencia ocurrirá a través del sistema de archivos de red o tal vez [rsync](https://en.wikipedia.org/wiki/Rsync)). - -Para obtener más información sobre las consultas relacionadas con las manipulaciones de particiones, consulte [Documentación de ALTER](../sql-reference/statements/alter.md#alter_manipulations-with-partitions). - -Una herramienta de terceros está disponible para automatizar este enfoque: [Haga clic en el botón de copia de seguridad](https://github.com/AlexAkulov/clickhouse-backup). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/backup/) diff --git a/docs/es/operations/configuration-files.md b/docs/es/operations/configuration-files.md deleted file mode 100644 index d9aa8567868..00000000000 --- a/docs/es/operations/configuration-files.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 50 -toc_title: "Archivos de configuraci\xF3n" ---- - -# Archivos de configuración {#configuration_files} - -ClickHouse admite la administración de configuración de varios archivos. El archivo de configuración del servidor principal es `/etc/clickhouse-server/config.xml`. Otros archivos deben estar en el `/etc/clickhouse-server/config.d` directorio. - -!!! note "Nota" - Todos los archivos de configuración deben estar en formato XML. Además, deben tener el mismo elemento raíz, generalmente ``. - -Algunos valores especificados en el archivo de configuración principal se pueden anular en otros archivos de configuración. El `replace` o `remove` se pueden especificar atributos para los elementos de estos archivos de configuración. - -Si no se especifica ninguno, combina el contenido de los elementos de forma recursiva, reemplazando los valores de los elementos secundarios duplicados. - -Si `replace` se especifica, reemplaza todo el elemento por el especificado. - -Si `remove` se especifica, elimina el elemento. - -La configuración también puede definir “substitutions”. Si un elemento tiene el `incl` atributo, la sustitución correspondiente del archivo se utilizará como el valor. De forma predeterminada, la ruta al archivo con sustituciones es `/etc/metrika.xml`. Esto se puede cambiar en el [include_from](server-configuration-parameters/settings.md#server_configuration_parameters-include_from) elemento en la configuración del servidor. Los valores de sustitución se especifican en `/yandex/substitution_name` elementos en este archivo. Si una sustitución especificada en `incl` no existe, se registra en el registro. Para evitar que ClickHouse registre las sustituciones que faltan, especifique `optional="true"` atributo (por ejemplo, ajustes para [macro](server-configuration-parameters/settings.md)). - -Las sustituciones también se pueden realizar desde ZooKeeper. Para hacer esto, especifique el atributo `from_zk = "/path/to/node"`. El valor del elemento se sustituye por el contenido del nodo en `/path/to/node` en ZooKeeper. También puede colocar un subárbol XML completo en el nodo ZooKeeper y se insertará completamente en el elemento de origen. - -El `config.xml` file puede especificar una configuración separada con configuraciones de usuario, perfiles y cuotas. La ruta relativa a esta configuración se establece en el `users_config` elemento. Por defecto, es `users.xml`. Si `users_config` se omite, la configuración de usuario, los perfiles y las cuotas se especifican directamente en `config.xml`. - -La configuración de los usuarios se puede dividir en archivos separados similares a `config.xml` y `config.d/`. -El nombre del directorio se define como `users_config` sin `.xml` postfix concatenado con `.d`. -Directorio `users.d` se utiliza por defecto, como `users_config` por defecto `users.xml`. -Por ejemplo, puede tener un archivo de configuración separado para cada usuario como este: - -``` bash -$ cat /etc/clickhouse-server/users.d/alice.xml -``` - -``` xml - - - - analytics - - ::/0 - - ... - analytics - - - -``` - -Para cada archivo de configuración, el servidor también genera `file-preprocessed.xml` archivos al iniciar. Estos archivos contienen todas las sustituciones y anulaciones completadas, y están destinados para uso informativo. Si se utilizaron sustituciones de ZooKeeper en los archivos de configuración pero ZooKeeper no está disponible en el inicio del servidor, el servidor carga la configuración desde el archivo preprocesado. - -El servidor realiza un seguimiento de los cambios en los archivos de configuración, así como archivos y nodos ZooKeeper que se utilizaron al realizar sustituciones y anulaciones, y vuelve a cargar la configuración de los usuarios y clústeres sobre la marcha. Esto significa que puede modificar el clúster, los usuarios y su configuración sin reiniciar el servidor. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/configuration_files/) diff --git a/docs/es/operations/index.md b/docs/es/operations/index.md deleted file mode 100644 index 9a928fa0f01..00000000000 --- a/docs/es/operations/index.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Operaci\xF3n" -toc_priority: 41 -toc_title: "Implantaci\xF3n" ---- - -# Operación {#operations} - -El manual de operaciones de ClickHouse consta de las siguientes secciones principales: - -- [Requisito](requirements.md) -- [Monitoreo](monitoring.md) -- [Solución de problemas](troubleshooting.md) -- [Recomendaciones de uso](tips.md) -- [Procedimiento de actualización](update.md) -- [Derechos de acceso](access-rights.md) -- [Copia de seguridad de datos](backup.md) -- [Archivos de configuración](configuration-files.md) -- [Cuota](quotas.md) -- [Tablas del sistema](system-tables.md) -- [Parámetros de configuración del servidor](server-configuration-parameters/index.md) -- [Cómo probar su hardware con ClickHouse](performance-test.md) -- [Configuración](settings/index.md) -- [Utilidad](utilities/index.md) - -{## [Artículo Original](https://clickhouse.tech/docs/en/operations/) ##} diff --git a/docs/es/operations/monitoring.md b/docs/es/operations/monitoring.md deleted file mode 100644 index 19912d23f3b..00000000000 --- a/docs/es/operations/monitoring.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: Monitoreo ---- - -# Monitoreo {#monitoring} - -Usted puede monitorear: - -- Utilización de recursos de hardware. -- Métricas del servidor ClickHouse. - -## Utilización de recursos {#resource-utilization} - -ClickHouse no supervisa el estado de los recursos de hardware por sí mismo. - -Se recomienda encarecidamente configurar la supervisión para: - -- Carga y temperatura en los procesadores. - - Usted puede utilizar [dmesg](https://en.wikipedia.org/wiki/Dmesg), [Turbostat](https://www.linux.org/docs/man8/turbostat.html) u otros instrumentos. - -- Utilización del sistema de almacenamiento, RAM y red. - -## Métricas del servidor ClickHouse {#clickhouse-server-metrics} - -El servidor ClickHouse tiene instrumentos integrados para el monitoreo de estado propio. - -Para realizar un seguimiento de los eventos del servidor, use los registros del servidor. Ver el [registrador](server-configuration-parameters/settings.md#server_configuration_parameters-logger) sección del archivo de configuración. - -ClickHouse recoge: - -- Diferentes métricas de cómo el servidor utiliza recursos computacionales. -- Estadísticas comunes sobre el procesamiento de consultas. - -Puede encontrar métricas en el [sistema.métricas](../operations/system-tables.md#system_tables-metrics), [sistema.evento](../operations/system-tables.md#system_tables-events), y [sistema.asynchronous_metrics](../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. - -Puede configurar ClickHouse para exportar métricas a [Grafito](https://github.com/graphite-project). Ver el [Sección de grafito](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) en el archivo de configuración del servidor ClickHouse. Antes de configurar la exportación de métricas, debe configurar Graphite siguiendo sus [guiar](https://graphite.readthedocs.io/en/latest/install.html). - -Puede configurar ClickHouse para exportar métricas a [Prometeo](https://prometheus.io). Ver el [Sección Prometheus](server-configuration-parameters/settings.md#server_configuration_parameters-prometheus) en el archivo de configuración del servidor ClickHouse. Antes de configurar la exportación de métricas, debe configurar Prometheus siguiendo su oficial [guiar](https://prometheus.io/docs/prometheus/latest/installation/). - -Además, puede supervisar la disponibilidad del servidor a través de la API HTTP. Enviar el `HTTP GET` solicitud de `/ping`. Si el servidor está disponible, responde con `200 OK`. - -Para supervisar servidores en una configuración de clúster, debe establecer [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parámetro y utilizar el recurso HTTP `/replicas_status`. Una solicitud para `/replicas_status` devoluciones `200 OK` si la réplica está disponible y no se retrasa detrás de las otras réplicas. Si una réplica se retrasa, devuelve `503 HTTP_SERVICE_UNAVAILABLE` con información sobre la brecha. diff --git a/docs/es/operations/optimizing-performance/index.md b/docs/es/operations/optimizing-performance/index.md deleted file mode 100644 index d2796c6e0d3..00000000000 --- a/docs/es/operations/optimizing-performance/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Optimizaci\xF3n del rendimiento" -toc_priority: 52 ---- - - diff --git a/docs/es/operations/optimizing-performance/sampling-query-profiler.md b/docs/es/operations/optimizing-performance/sampling-query-profiler.md deleted file mode 100644 index a474dde6af2..00000000000 --- a/docs/es/operations/optimizing-performance/sampling-query-profiler.md +++ /dev/null @@ -1,64 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 54 -toc_title: "Generaci\xF3n de perfiles de consultas" ---- - -# Analizador de consultas de muestreo {#sampling-query-profiler} - -ClickHouse ejecuta el generador de perfiles de muestreo que permite analizar la ejecución de consultas. Utilizando el generador de perfiles puede encontrar rutinas de código fuente que se utilizan con más frecuencia durante la ejecución de la consulta. Puede rastrear el tiempo de CPU y el tiempo de reloj de pared invertido, incluido el tiempo de inactividad. - -Para usar el generador de perfiles: - -- Configurar el [trace_log](../server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) sección de la configuración del servidor. - - Esta sección configura la [trace_log](../../operations/system-tables.md#system_tables-trace_log) tabla del sistema que contiene los resultados del funcionamiento del generador de perfiles. Está configurado de forma predeterminada. Recuerde que los datos de esta tabla solo son válidos para un servidor en ejecución. Después de reiniciar el servidor, ClickHouse no limpia la tabla y toda la dirección de memoria virtual almacenada puede dejar de ser válida. - -- Configurar el [Los resultados de la prueba](../settings/settings.md#query_profiler_cpu_time_period_ns) o [query_profiler_real_time_period_ns](../settings/settings.md#query_profiler_real_time_period_ns) configuración. Ambos ajustes se pueden utilizar simultáneamente. - - Estas opciones le permiten configurar temporizadores del generador de perfiles. Como estos son los ajustes de sesión, puede obtener diferentes frecuencias de muestreo para todo el servidor, usuarios individuales o perfiles de usuario, para su sesión interactiva y para cada consulta individual. - -La frecuencia de muestreo predeterminada es una muestra por segundo y tanto la CPU como los temporizadores reales están habilitados. Esta frecuencia permite recopilar suficiente información sobre el clúster ClickHouse. Al mismo tiempo, al trabajar con esta frecuencia, el generador de perfiles no afecta el rendimiento del servidor ClickHouse. Si necesita perfilar cada consulta individual, intente usar una mayor frecuencia de muestreo. - -Para analizar el `trace_log` tabla del sistema: - -- Instale el `clickhouse-common-static-dbg` paquete. Ver [Instalar desde paquetes DEB](../../getting-started/install.md#install-from-deb-packages). - -- Permitir funciones de introspección [allow_introspection_functions](../settings/settings.md#settings-allow_introspection_functions) configuración. - - Por razones de seguridad, las funciones de introspección están deshabilitadas de forma predeterminada. - -- Utilice el `addressToLine`, `addressToSymbol` y `demangle` [funciones de la introspección](../../sql-reference/functions/introspection.md) para obtener nombres de funciones y sus posiciones en el código ClickHouse. Para obtener un perfil para alguna consulta, debe agregar datos del `trace_log` tabla. Puede agregar datos por funciones individuales o por los seguimientos de pila completos. - -Si necesita visualizar `trace_log` información, intente [Flamegraph](../../interfaces/third-party/gui/#clickhouse-flamegraph) y [Nivel de Cifrado WEP](https://github.com/laplab/clickhouse-speedscope). - -## Ejemplo {#example} - -En este ejemplo nos: - -- Filtrado `trace_log` datos por un identificador de consulta y la fecha actual. - -- Agregando por seguimiento de pila. - -- Usando funciones de introspección, obtendremos un informe de: - - - Nombres de símbolos y funciones de código fuente correspondientes. - - Ubicaciones del código fuente de estas funciones. - - - -``` sql -SELECT - count(), - arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym -FROM system.trace_log -WHERE (query_id = 'ebca3574-ad0a-400a-9cbc-dca382f5998c') AND (event_date = today()) -GROUP BY trace -ORDER BY count() DESC -LIMIT 10 -``` - -``` text -{% include "examples/sampling_query_profiler_result.txt" %} -``` diff --git a/docs/es/operations/performance-test.md b/docs/es/operations/performance-test.md deleted file mode 100644 index 97444f339cd..00000000000 --- a/docs/es/operations/performance-test.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 54 -toc_title: Prueba de hardware ---- - -# Cómo probar su hardware con ClickHouse {#how-to-test-your-hardware-with-clickhouse} - -Con esta instrucción, puede ejecutar una prueba de rendimiento básica de ClickHouse en cualquier servidor sin instalar paquetes de ClickHouse. - -1. Ir a “commits” página: https://github.com/ClickHouse/ClickHouse/commits/master - -2. Haga clic en la primera marca de verificación verde o cruz roja con verde “ClickHouse Build Check” y haga clic en el “Details” enlace cerca “ClickHouse Build Check”. No existe tal enlace en algunas confirmaciones, por ejemplo, confirmaciones con documentación. En este caso, elija la confirmación más cercana que tenga este enlace. - -3. Copie el enlace a “clickhouse” binario para amd64 o aarch64. - -4. ssh al servidor y descargarlo con wget: - - - - # For amd64: - wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578163263_binary/clickhouse - # For aarch64: - wget https://clickhouse-builds.s3.yandex.net/0/00ba767f5d2a929394ea3be193b1f79074a1c4bc/1578161264_binary/clickhouse - # Then do: - chmod a+x clickhouse - -1. Descargar configs: - - - - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.xml - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/users.xml - mkdir config.d - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/path.xml -O config.d/path.xml - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/log_to_console.xml -O config.d/log_to_console.xml - -1. Descargar archivos de referencia: - - - - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/benchmark-new.sh - chmod a+x benchmark-new.sh - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/benchmark/clickhouse/queries.sql - -1. Descargue los datos de prueba de acuerdo con el [El Yandex.Conjunto de datos de Metrica](../getting-started/example-datasets/metrica.md) instrucción (“hits” tabla que contiene 100 millones de filas). - - - - wget https://datasets.clickhouse.tech/hits/partitions/hits_100m_obfuscated_v1.tar.xz - tar xvf hits_100m_obfuscated_v1.tar.xz -C . - mv hits_100m_obfuscated_v1/* . - -1. Ejecute el servidor: - - - - ./clickhouse server - -1. Verifique los datos: ssh al servidor en otro terminal - - - - ./clickhouse client --query "SELECT count() FROM hits_100m_obfuscated" - 100000000 - -1. Edite el benchmark-new.sh, cambie `clickhouse-client` a `./clickhouse client` y añadir `–-max_memory_usage 100000000000` parámetro. - - - - mcedit benchmark-new.sh - -1. Ejecute el punto de referencia: - - - - ./benchmark-new.sh hits_100m_obfuscated - -1. Envíe los números y la información sobre la configuración de su hardware a clickhouse-feedback@yandex-team.com - -Todos los resultados se publican aquí: https://clickhouse.tecnología/punto de referencia/hardware/ diff --git a/docs/es/operations/quotas.md b/docs/es/operations/quotas.md deleted file mode 100644 index 9d84ce21339..00000000000 --- a/docs/es/operations/quotas.md +++ /dev/null @@ -1,112 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 51 -toc_title: Cuota ---- - -# Cuota {#quotas} - -Las cuotas le permiten limitar el uso de recursos durante un período de tiempo o realizar un seguimiento del uso de recursos. -Las cuotas se configuran en la configuración del usuario, que generalmente ‘users.xml’. - -El sistema también tiene una característica para limitar la complejidad de una sola consulta. Vea la sección “Restrictions on query complexity”). - -A diferencia de las restricciones de complejidad de consultas, las cuotas: - -- Coloque restricciones en un conjunto de consultas que se pueden ejecutar durante un período de tiempo, en lugar de limitar una sola consulta. -- Tenga en cuenta los recursos gastados en todos los servidores remotos para el procesamiento de consultas distribuidas. - -Veamos la sección del ‘users.xml’ fichero que define las cuotas. - -``` xml - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - -``` - -De forma predeterminada, la cuota realiza un seguimiento del consumo de recursos para cada hora, sin limitar el uso. -El consumo de recursos calculado para cada intervalo se envía al registro del servidor después de cada solicitud. - -``` xml - - - - - 3600 - - 1000 - 100 - 1000000000 - 100000000000 - 900 - - - - 86400 - - 10000 - 1000 - 5000000000 - 500000000000 - 7200 - - -``` - -Para el ‘statbox’ Las restricciones se establecen por cada hora y por cada 24 horas (86.400 segundos). El intervalo de tiempo se cuenta, a partir de un momento fijo definido por la implementación en el tiempo. En otras palabras, el intervalo de 24 horas no necesariamente comienza a medianoche. - -Cuando finaliza el intervalo, se borran todos los valores recopilados. Para la siguiente hora, el cálculo de la cuota comienza de nuevo. - -Estas son las cantidades que se pueden restringir: - -`queries` – The total number of requests. - -`errors` – The number of queries that threw an exception. - -`result_rows` – The total number of rows given as a result. - -`read_rows` – The total number of source rows read from tables for running the query on all remote servers. - -`execution_time` – The total query execution time, in seconds (wall time). - -Si se excede el límite durante al menos un intervalo de tiempo, se lanza una excepción con un texto sobre qué restricción se excedió, para qué intervalo y cuándo comienza el nuevo intervalo (cuando se pueden enviar consultas nuevamente). - -Las cuotas pueden usar el “quota key” característica para informar sobre los recursos para múltiples claves de forma independiente. Aquí hay un ejemplo de esto: - -``` xml - - - - -``` - -La cuota se asigna a los usuarios ‘users’ sección de la configuración. Vea la sección “Access rights”. - -Para el procesamiento de consultas distribuidas, los importes acumulados se almacenan en el servidor del solicitante. Entonces, si el usuario va a otro servidor, la cuota allí “start over”. - -Cuando se reinicia el servidor, las cuotas se restablecen. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/quotas/) diff --git a/docs/es/operations/requirements.md b/docs/es/operations/requirements.md deleted file mode 100644 index d6f0f25cf21..00000000000 --- a/docs/es/operations/requirements.md +++ /dev/null @@ -1,61 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: Requisito ---- - -# Requisito {#requirements} - -## CPU {#cpu} - -Para la instalación desde paquetes deb precompilados, utilice una CPU con arquitectura x86_64 y soporte para las instrucciones de SSE 4.2. Para ejecutar ClickHouse con procesadores que no admiten SSE 4.2 o tienen arquitectura AArch64 o PowerPC64LE, debe compilar ClickHouse a partir de fuentes. - -ClickHouse implementa el procesamiento de datos paralelo y utiliza todos los recursos de hardware disponibles. Al elegir un procesador, tenga en cuenta que ClickHouse funciona de manera más eficiente en configuraciones con un gran número de núcleos pero con una velocidad de reloj más baja que en configuraciones con menos núcleos y una velocidad de reloj más alta. Por ejemplo, 16 núcleos con 2600 MHz es preferible a 8 núcleos con 3600 MHz. - -Se recomienda usar **Impulso de Turbo** y **hiper-threading** tecnología. Mejora significativamente el rendimiento con una carga de trabajo típica. - -## RAM {#ram} - -Recomendamos utilizar un mínimo de 4 GB de RAM para realizar consultas no triviales. El servidor ClickHouse puede ejecutarse con una cantidad mucho menor de RAM, pero requiere memoria para procesar consultas. - -El volumen requerido de RAM depende de: - -- La complejidad de las consultas. -- La cantidad de datos que se procesan en las consultas. - -Para calcular el volumen requerido de RAM, debe estimar el tamaño de los datos temporales para [GROUP BY](../sql-reference/statements/select/group-by.md#select-group-by-clause), [DISTINCT](../sql-reference/statements/select/distinct.md#select-distinct), [JOIN](../sql-reference/statements/select/join.md#select-join) y otras operaciones que utilice. - -ClickHouse puede usar memoria externa para datos temporales. Ver [GROUP BY en memoria externa](../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory) para más detalles. - -## Archivo de intercambio {#swap-file} - -Deshabilite el archivo de intercambio para entornos de producción. - -## Subsistema de almacenamiento {#storage-subsystem} - -Necesita tener 2 GB de espacio libre en disco para instalar ClickHouse. - -El volumen de almacenamiento requerido para sus datos debe calcularse por separado. La evaluación debe incluir: - -- Estimación del volumen de datos. - - Puede tomar una muestra de los datos y obtener el tamaño promedio de una fila de ella. Luego multiplique el valor por el número de filas que planea almacenar. - -- El coeficiente de compresión de datos. - - Para estimar el coeficiente de compresión de datos, cargue una muestra de sus datos en ClickHouse y compare el tamaño real de los datos con el tamaño de la tabla almacenada. Por ejemplo, los datos de clickstream generalmente se comprimen de 6 a 10 veces. - -Para calcular el volumen final de datos que se almacenarán, aplique el coeficiente de compresión al volumen de datos estimado. Si planea almacenar datos en varias réplicas, multiplique el volumen estimado por el número de réplicas. - -## Red {#network} - -Si es posible, use redes de 10G o clase superior. - -El ancho de banda de la red es fundamental para procesar consultas distribuidas con una gran cantidad de datos intermedios. Además, la velocidad de la red afecta a los procesos de replicación. - -## Software {#software} - -ClickHouse está desarrollado principalmente para la familia de sistemas operativos Linux. La distribución de Linux recomendada es Ubuntu. El `tzdata` paquete debe ser instalado en el sistema. - -ClickHouse también puede funcionar en otras familias de sistemas operativos. Ver detalles en el [Primeros pasos](../getting-started/index.md) sección de la documentación. diff --git a/docs/es/operations/server-configuration-parameters/index.md b/docs/es/operations/server-configuration-parameters/index.md deleted file mode 100644 index e1e2e777b94..00000000000 --- a/docs/es/operations/server-configuration-parameters/index.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Par\xE1metros de configuraci\xF3n del servidor" -toc_priority: 54 -toc_title: "Implantaci\xF3n" ---- - -# Parámetros de configuración del servidor {#server-settings} - -Esta sección contiene descripciones de la configuración del servidor que no se puede cambiar en el nivel de sesión o consulta. - -Estos ajustes se almacenan en el `config.xml` archivo en el servidor ClickHouse. - -Otros ajustes se describen en el “[Configuración](../settings/index.md#session-settings-intro)” apartado. - -Antes de estudiar la configuración, lea el [Archivos de configuración](../configuration-files.md#configuration_files) sección y tomar nota del uso de sustituciones (el `incl` y `optional` atributo). - -[Artículo Original](https://clickhouse.tech/docs/en/operations/server_configuration_parameters/) diff --git a/docs/es/operations/server-configuration-parameters/settings.md b/docs/es/operations/server-configuration-parameters/settings.md deleted file mode 100644 index 86264ed0440..00000000000 --- a/docs/es/operations/server-configuration-parameters/settings.md +++ /dev/null @@ -1,906 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 57 -toc_title: "Configuraci\xF3n del servidor" ---- - -# Configuración del servidor {#server-settings} - -## builtin_dictionaries_reload_interval {#builtin-dictionaries-reload-interval} - -El intervalo en segundos antes de volver a cargar los diccionarios integrados. - -ClickHouse recarga los diccionarios incorporados cada x segundos. Esto hace posible editar diccionarios “on the fly” sin reiniciar el servidor. - -Valor predeterminado: 3600. - -**Ejemplo** - -``` xml -3600 -``` - -## compresión {#server-settings-compression} - -Ajustes de compresión de datos para [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)-mesas de motor. - -!!! warning "Advertencia" - No lo use si acaba de comenzar a usar ClickHouse. - -Plantilla de configuración: - -``` xml - - - ... - ... - ... - - ... - -``` - -`` campo: - -- `min_part_size` – The minimum size of a data part. -- `min_part_size_ratio` – The ratio of the data part size to the table size. -- `method` – Compression method. Acceptable values: `lz4` o `zstd`. - -Puede configurar múltiples `` apartado. - -Acciones cuando se cumplen las condiciones: - -- Si un elemento de datos coincide con un conjunto de condiciones, ClickHouse utiliza el método de compresión especificado. -- Si un elemento de datos coincide con varios conjuntos de condiciones, ClickHouse utiliza el primer conjunto de condiciones coincidente. - -Si no se cumplen condiciones para un elemento de datos, ClickHouse utiliza el `lz4` compresión. - -**Ejemplo** - -``` xml - - - 10000000000 - 0.01 - zstd - - -``` - -## default_database {#default-database} - -La base de datos predeterminada. - -Para obtener una lista de bases de datos, [SHOW DATABASES](../../sql-reference/statements/show.md#show-databases) consulta. - -**Ejemplo** - -``` xml -default -``` - -## default_profile {#default-profile} - -Perfil de configuración predeterminado. - -Los perfiles de configuración se encuentran en el archivo especificado en el parámetro `user_config`. - -**Ejemplo** - -``` xml -default -``` - -## Diccionarios_config {#server_configuration_parameters-dictionaries_config} - -La ruta de acceso al archivo de configuración para diccionarios externos. - -Camino: - -- Especifique la ruta absoluta o la ruta relativa al archivo de configuración del servidor. -- La ruta puede contener comodines \* y ?. - -Ver también “[Diccionarios externos](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md)”. - -**Ejemplo** - -``` xml -*_dictionary.xml -``` - -## Diccionarios_lazy_load {#server_configuration_parameters-dictionaries_lazy_load} - -La carga perezosa de los diccionarios. - -Si `true`, entonces cada diccionario es creado en el primer uso. Si se produce un error en la creación del diccionario, la función que estaba utilizando el diccionario produce una excepción. - -Si `false`, todos los diccionarios se crean cuando se inicia el servidor, y si hay un error, el servidor se apaga. - -El valor predeterminado es `true`. - -**Ejemplo** - -``` xml -true -``` - -## format_schema_path {#server_configuration_parameters-format_schema_path} - -La ruta de acceso al directorio con los esquemas para los datos de entrada, como los esquemas [CapnProto](../../interfaces/formats.md#capnproto) formato. - -**Ejemplo** - -``` xml - - format_schemas/ -``` - -## grafito {#server_configuration_parameters-graphite} - -Envío de datos a [Grafito](https://github.com/graphite-project). - -Configuración: - -- host – The Graphite server. -- port – The port on the Graphite server. -- interval – The interval for sending, in seconds. -- timeout – The timeout for sending data, in seconds. -- root_path – Prefix for keys. -- metrics – Sending data from the [sistema.métricas](../../operations/system-tables.md#system_tables-metrics) tabla. -- events – Sending deltas data accumulated for the time period from the [sistema.evento](../../operations/system-tables.md#system_tables-events) tabla. -- events_cumulative – Sending cumulative data from the [sistema.evento](../../operations/system-tables.md#system_tables-events) tabla. -- asynchronous_metrics – Sending data from the [sistema.asynchronous_metrics](../../operations/system-tables.md#system_tables-asynchronous_metrics) tabla. - -Puede configurar múltiples `` clausula. Por ejemplo, puede usar esto para enviar datos diferentes a intervalos diferentes. - -**Ejemplo** - -``` xml - - localhost - 42000 - 0.1 - 60 - one_min - true - true - false - true - -``` - -## graphite_rollup {#server_configuration_parameters-graphite-rollup} - -Ajustes para reducir los datos de grafito. - -Para obtener más información, consulte [GraphiteMergeTree](../../engines/table-engines/mergetree-family/graphitemergetree.md). - -**Ejemplo** - -``` xml - - - max - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - -``` - -## http_port/https_port {#http-porthttps-port} - -El puerto para conectarse al servidor a través de HTTP(s). - -Si `https_port` se especifica, [openSSL](#server_configuration_parameters-openssl) debe ser configurado. - -Si `http_port` se especifica, la configuración de OpenSSL se ignora incluso si está establecida. - -**Ejemplo** - -``` xml -9999 -``` - -## http_server_default_response {#server_configuration_parameters-http_server_default_response} - -La página que se muestra de forma predeterminada al acceder al servidor HTTP de ClickHouse. -El valor predeterminado es “Ok.” (con un avance de línea al final) - -**Ejemplo** - -Abrir `https://tabix.io/` al acceder `http://localhost: http_port`. - -``` xml - -
]]> -
-``` - -## include_from {#server_configuration_parameters-include_from} - -La ruta al archivo con sustituciones. - -Para obtener más información, consulte la sección “[Archivos de configuración](../configuration-files.md#configuration_files)”. - -**Ejemplo** - -``` xml -/etc/metrica.xml -``` - -## Interesante {#interserver-http-port} - -Puerto para el intercambio de datos entre servidores ClickHouse. - -**Ejemplo** - -``` xml -9009 -``` - -## Sistema abierto {#interserver-http-host} - -El nombre de host que pueden utilizar otros servidores para acceder a este servidor. - -Si se omite, se define de la misma manera que el `hostname-f` comando. - -Útil para separarse de una interfaz de red específica. - -**Ejemplo** - -``` xml -example.yandex.ru -``` - -## interserver_http_credentials {#server-settings-interserver-http-credentials} - -El nombre de usuario y la contraseña utilizados para [replicación](../../engines/table-engines/mergetree-family/replication.md) con los motores Replicated\*. Estas credenciales sólo se utilizan para la comunicación entre réplicas y no están relacionadas con las credenciales de los clientes de ClickHouse. El servidor está comprobando estas credenciales para conectar réplicas y utiliza las mismas credenciales cuando se conecta a otras réplicas. Por lo tanto, estas credenciales deben establecerse igual para todas las réplicas de un clúster. -De forma predeterminada, la autenticación no se utiliza. - -Esta sección contiene los siguientes parámetros: - -- `user` — username. -- `password` — password. - -**Ejemplo** - -``` xml - - admin - 222 - -``` - -## keep_alive_timeout {#keep-alive-timeout} - -El número de segundos que ClickHouse espera las solicitudes entrantes antes de cerrar la conexión. El valor predeterminado es de 3 segundos. - -**Ejemplo** - -``` xml -3 -``` - -## listen_host {#server_configuration_parameters-listen_host} - -Restricción en hosts de los que pueden provenir las solicitudes. Si desea que el servidor responda a todos ellos, especifique `::`. - -Ejemplos: - -``` xml -::1 -127.0.0.1 -``` - -## registrador {#server_configuration_parameters-logger} - -Configuración de registro. - -Claves: - -- level – Logging level. Acceptable values: `trace`, `debug`, `information`, `warning`, `error`. -- log – The log file. Contains all the entries according to `level`. -- errorlog – Error log file. -- size – Size of the file. Applies to `log`y`errorlog`. Una vez que el archivo alcanza `size`, ClickHouse archiva y cambia el nombre, y crea un nuevo archivo de registro en su lugar. -- count – The number of archived log files that ClickHouse stores. - -**Ejemplo** - -``` xml - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - -``` - -También se admite la escritura en el syslog. Config ejemplo: - -``` xml - - 1 - -
syslog.remote:10514
- myhost.local - LOG_LOCAL6 - syslog -
-
-``` - -Claves: - -- use_syslog — Required setting if you want to write to the syslog. -- address — The host\[:port\] of syslogd. If omitted, the local daemon is used. -- hostname — Optional. The name of the host that logs are sent from. -- facility — [La palabra clave syslog facility](https://en.wikipedia.org/wiki/Syslog#Facility) en letras mayúsculas con el “LOG_” prefijo: (`LOG_USER`, `LOG_DAEMON`, `LOG_LOCAL3` y así sucesivamente). - Valor predeterminado: `LOG_USER` si `address` se especifica, `LOG_DAEMON otherwise.` -- format – Message format. Possible values: `bsd` y `syslog.` - -## macro {#macros} - -Sustituciones de parámetros para tablas replicadas. - -Se puede omitir si no se utilizan tablas replicadas. - -Para obtener más información, consulte la sección “[Creación de tablas replicadas](../../engines/table-engines/mergetree-family/replication.md)”. - -**Ejemplo** - -``` xml - -``` - -## Método de codificación de datos: {#server-mark-cache-size} - -Tamaño aproximado (en bytes) de la memoria caché de marcas utilizadas por los motores de [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md) familia. - -La memoria caché se comparte para el servidor y la memoria se asigna según sea necesario. El tamaño de la memoria caché debe ser al menos 5368709120. - -**Ejemplo** - -``` xml -5368709120 -``` - -## max_concurrent_queries {#max-concurrent-queries} - -El número máximo de solicitudes procesadas simultáneamente. - -**Ejemplo** - -``` xml -100 -``` - -## max_connections {#max-connections} - -El número máximo de conexiones entrantes. - -**Ejemplo** - -``` xml -4096 -``` - -## max_open_files {#max-open-files} - -El número máximo de archivos abiertos. - -Predeterminada: `maximum`. - -Recomendamos usar esta opción en Mac OS X desde el `getrlimit()` función devuelve un valor incorrecto. - -**Ejemplo** - -``` xml -262144 -``` - -## max_table_size_to_drop {#max-table-size-to-drop} - -Restricción en la eliminación de tablas. - -Si el tamaño de un [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md) mesa excede `max_table_size_to_drop` (en bytes), no puede eliminarlo usando una consulta DROP. - -Si aún necesita eliminar la tabla sin reiniciar el servidor ClickHouse, cree el `/flags/force_drop_table` y ejecute la consulta DROP. - -Valor predeterminado: 50 GB. - -El valor 0 significa que puede eliminar todas las tablas sin restricciones. - -**Ejemplo** - -``` xml -0 -``` - -## merge_tree {#server_configuration_parameters-merge_tree} - -Ajuste fino para tablas en el [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). - -Para obtener más información, vea MergeTreeSettings.h archivo de encabezado. - -**Ejemplo** - -``` xml - - 5 - -``` - -## openSSL {#server_configuration_parameters-openssl} - -Configuración cliente/servidor SSL. - -El soporte para SSL es proporcionado por el `libpoco` biblioteca. La interfaz se describe en el archivo [Nombre de la red inalámbrica (SSID):h](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) - -Claves para la configuración del servidor/cliente: - -- privateKeyFile – The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time. -- certificateFile – The path to the client/server certificate file in PEM format. You can omit it if `privateKeyFile` contiene el certificado. -- caConfig – The path to the file or directory that contains trusted root certificates. -- verificationMode – The method for checking the node's certificates. Details are in the description of the [Contexto](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) clase. Valores posibles: `none`, `relaxed`, `strict`, `once`. -- verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. -- loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. -- cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Valores aceptables: `true`, `false`. -- sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. Este parámetro siempre se recomienda ya que ayuda a evitar problemas tanto si el servidor almacena en caché la sesión como si el cliente solicita el almacenamiento en caché. Valor predeterminado: `${application.name}`. -- sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. -- sessionTimeout – Time for caching the session on the server. -- extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. -- requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. -- requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. -- fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. -- privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . -- disableProtocols – Protocols that are not allowed to use. -- preferServerCiphers – Preferred server ciphers on the client. - -**Ejemplo de configuración:** - -``` xml - - - - /etc/clickhouse-server/server.crt - /etc/clickhouse-server/server.key - - /etc/clickhouse-server/dhparam.pem - none - true - true - sslv2,sslv3 - true - - - true - true - sslv2,sslv3 - true - - - - RejectCertificateHandler - - - -``` - -## part_log {#server_configuration_parameters-part-log} - -Registro de eventos asociados con [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). Por ejemplo, agregar o fusionar datos. Puede utilizar el registro para simular algoritmos de combinación y comparar sus características. Puede visualizar el proceso de fusión. - -Las consultas se registran en el [sistema.part_log](../../operations/system-tables.md#system_tables-part-log) tabla, no en un archivo separado. Puede configurar el nombre de esta tabla en el `table` parámetro (ver más abajo). - -Utilice los siguientes parámetros para configurar el registro: - -- `database` – Name of the database. -- `table` – Name of the system table. -- `partition_by` – Sets a [clave de partición personalizada](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. - -**Ejemplo** - -``` xml - - system - part_log
- toMonday(event_date) - 7500 -
-``` - -## camino {#server_configuration_parameters-path} - -La ruta de acceso al directorio que contiene los datos. - -!!! note "Nota" - La barra diagonal es obligatoria. - -**Ejemplo** - -``` xml -/var/lib/clickhouse/ -``` - -## prometeo {#server_configuration_parameters-prometheus} - -Exponer datos de métricas para raspar desde [Prometeo](https://prometheus.io). - -Configuración: - -- `endpoint` – HTTP endpoint for scraping metrics by prometheus server. Start from ‘/’. -- `port` – Port for `endpoint`. -- `metrics` – Flag that sets to expose metrics from the [sistema.métricas](../system-tables.md#system_tables-metrics) tabla. -- `events` – Flag that sets to expose metrics from the [sistema.evento](../system-tables.md#system_tables-events) tabla. -- `asynchronous_metrics` – Flag that sets to expose current metrics values from the [sistema.asynchronous_metrics](../system-tables.md#system_tables-asynchronous_metrics) tabla. - -**Ejemplo** - -``` xml - - /metrics - 8001 - true - true - true - -``` - -## query_log {#server_configuration_parameters-query-log} - -Configuración de las consultas de registro recibidas con [log_queries=1](../settings/settings.md) configuración. - -Las consultas se registran en el [sistema.query_log](../../operations/system-tables.md#system_tables-query_log) tabla, no en un archivo separado. Puede cambiar el nombre de la tabla en el `table` parámetro (ver más abajo). - -Utilice los siguientes parámetros para configurar el registro: - -- `database` – Name of the database. -- `table` – Name of the system table the queries will be logged in. -- `partition_by` – Sets a [clave de partición personalizada](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) para una mesa. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. - -Si la tabla no existe, ClickHouse la creará. Si la estructura del registro de consultas cambió cuando se actualizó el servidor ClickHouse, se cambia el nombre de la tabla con la estructura anterior y se crea una nueva tabla automáticamente. - -**Ejemplo** - -``` xml - - system - query_log
- toMonday(event_date) - 7500 -
-``` - -## Sistema abierto {#server_configuration_parameters-query-thread-log} - -Configuración de subprocesos de registro de consultas recibidas con [Log_query_threads = 1](../settings/settings.md#settings-log-query-threads) configuración. - -Las consultas se registran en el [sistema.Sistema abierto.](../../operations/system-tables.md#system_tables-query-thread-log) tabla, no en un archivo separado. Puede cambiar el nombre de la tabla en el `table` parámetro (ver más abajo). - -Utilice los siguientes parámetros para configurar el registro: - -- `database` – Name of the database. -- `table` – Name of the system table the queries will be logged in. -- `partition_by` – Sets a [clave de partición personalizada](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) para una tabla del sistema. -- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. - -Si la tabla no existe, ClickHouse la creará. Si la estructura del registro de subprocesos de consulta cambió cuando se actualizó el servidor ClickHouse, se cambia el nombre de la tabla con la estructura anterior y se crea una nueva tabla automáticamente. - -**Ejemplo** - -``` xml - - system - query_thread_log
- toMonday(event_date) - 7500 -
-``` - -## trace_log {#server_configuration_parameters-trace_log} - -Ajustes para el [trace_log](../../operations/system-tables.md#system_tables-trace_log) operación de la tabla del sistema. - -Parámetros: - -- `database` — Database for storing a table. -- `table` — Table name. -- `partition_by` — [Clave de partición personalizada](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) para una tabla del sistema. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. - -El archivo de configuración del servidor predeterminado `config.xml` contiene la siguiente sección de configuración: - -``` xml - - system - trace_log
- toYYYYMM(event_date) - 7500 -
-``` - -## query_masking_rules {#query-masking-rules} - -Reglas basadas en Regexp, que se aplicarán a las consultas, así como a todos los mensajes de registro antes de almacenarlos en los registros del servidor, -`system.query_log`, `system.text_log`, `system.processes` tabla, y en los registros enviados al cliente. Eso permite prevenir -fuga de datos sensible de consultas SQL (como nombres, correos electrónicos, -identificadores o números de tarjetas de crédito) a los registros. - -**Ejemplo** - -``` xml - - - hide SSN - (^|\D)\d{3}-\d{2}-\d{4}($|\D) - 000-00-0000 - - -``` - -Campos de configuración: -- `name` - nombre de la regla (opcional) -- `regexp` - Expresión regular compatible con RE2 (obligatoria) -- `replace` - cadena de sustitución para datos confidenciales (opcional, por defecto - seis asteriscos) - -Las reglas de enmascaramiento se aplican a toda la consulta (para evitar fugas de datos confidenciales de consultas mal formadas / no analizables). - -`system.events` la tabla tiene contador `QueryMaskingRulesMatch` que tienen un número total de coincidencias de reglas de enmascaramiento de consultas. - -Para consultas distribuidas, cada servidor debe configurarse por separado; de lo contrario, las subconsultas pasan a otros -los nodos se almacenarán sin enmascarar. - -## remote_servers {#server-settings-remote-servers} - -Configuración de los clústeres utilizados por [Distribuido](../../engines/table-engines/special/distributed.md) motor de mesa y por el `cluster` función de la tabla. - -**Ejemplo** - -``` xml - -``` - -Para el valor de la `incl` atributo, consulte la sección “[Archivos de configuración](../configuration-files.md#configuration_files)”. - -**Ver también** - -- [skip_unavailable_shards](../settings/settings.md#settings-skip_unavailable_shards) - -## Zona horaria {#server_configuration_parameters-timezone} - -La zona horaria del servidor. - -Especificado como un identificador de la IANA para la zona horaria UTC o la ubicación geográfica (por ejemplo, África/Abidjan). - -La zona horaria es necesaria para las conversiones entre los formatos String y DateTime cuando los campos DateTime se envían al formato de texto (impreso en la pantalla o en un archivo) y cuando se obtiene DateTime de una cadena. Además, la zona horaria se usa en funciones que funcionan con la hora y la fecha si no recibieron la zona horaria en los parámetros de entrada. - -**Ejemplo** - -``` xml -Europe/Moscow -``` - -## Tcp_port {#server_configuration_parameters-tcp_port} - -Puerto para comunicarse con clientes a través del protocolo TCP. - -**Ejemplo** - -``` xml -9000 -``` - -## Tcp_port_secure {#server_configuration_parameters-tcp_port_secure} - -Puerto TCP para una comunicación segura con los clientes. Úselo con [OpenSSL](#server_configuration_parameters-openssl) configuración. - -**Valores posibles** - -Entero positivo. - -**Valor predeterminado** - -``` xml -9440 -``` - -## mysql_port {#server_configuration_parameters-mysql_port} - -Puerto para comunicarse con clientes a través del protocolo MySQL. - -**Valores posibles** - -Entero positivo. - -Ejemplo - -``` xml -9004 -``` - -## tmp_path {#server-settings-tmp_path} - -Ruta de acceso a datos temporales para procesar consultas grandes. - -!!! note "Nota" - La barra diagonal es obligatoria. - -**Ejemplo** - -``` xml -/var/lib/clickhouse/tmp/ -``` - -## tmp_policy {#server-settings-tmp-policy} - -Política de [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) para almacenar archivos temporales. -Si no se establece [`tmp_path`](#server-settings-tmp_path) se utiliza, de lo contrario se ignora. - -!!! note "Nota" - - `move_factor` se ignora -- `keep_free_space_bytes` se ignora -- `max_data_part_size_bytes` se ignora -- debe tener exactamente un volumen en esa política - -## Uncompressed_cache_size {#server-settings-uncompressed_cache_size} - -Tamaño de la memoria caché (en bytes) para los datos sin comprimir utilizados por los motores de [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). - -Hay una caché compartida para el servidor. La memoria se asigna a pedido. La caché se usa si la opción [Use_uncompressed_cache](../settings/settings.md#setting-use_uncompressed_cache) está habilitado. - -La caché sin comprimir es ventajosa para consultas muy cortas en casos individuales. - -**Ejemplo** - -``` xml -8589934592 -``` - -## user_files_path {#server_configuration_parameters-user_files_path} - -El directorio con archivos de usuario. Utilizado en la función de tabla [file()](../../sql-reference/table-functions/file.md). - -**Ejemplo** - -``` xml -/var/lib/clickhouse/user_files/ -``` - -## users_config {#users-config} - -Ruta de acceso al archivo que contiene: - -- Configuraciones de usuario. -- Derechos de acceso. -- Perfiles de configuración. -- Configuración de cuota. - -**Ejemplo** - -``` xml -users.xml -``` - -## Zookeeper {#server-settings_zookeeper} - -Contiene la configuración que permite a ClickHouse interactuar con [ZooKeeper](http://zookeeper.apache.org/) Cluster. - -ClickHouse utiliza ZooKeeper para almacenar metadatos de réplicas cuando se utilizan tablas replicadas. Si no se utilizan tablas replicadas, se puede omitir esta sección de parámetros. - -Esta sección contiene los siguientes parámetros: - -- `node` — ZooKeeper endpoint. You can set multiple endpoints. - - Por ejemplo: - - - -``` xml - - example_host - 2181 - -``` - - The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster. - -- `session_timeout` — Maximum timeout for the client session in milliseconds. -- `root` — The [Znode](http://zookeeper.apache.org/doc/r3.5.5/zookeeperOver.html#Nodes+and+ephemeral+nodes) que se utiliza como la raíz de los znodes utilizados por el servidor ClickHouse. Opcional. -- `identity` — User and password, that can be required by ZooKeeper to give access to requested znodes. Optional. - -**Ejemplo de configuración** - -``` xml - - - example1 - 2181 - - - example2 - 2181 - - 30000 - 10000 - - /path/to/zookeeper/node - - user:password - -``` - -**Ver también** - -- [Replicación](../../engines/table-engines/mergetree-family/replication.md) -- [Guía del programador ZooKeeper](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) - -## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} - -Método de almacenamiento para encabezados de parte de datos en ZooKeeper. - -Esta configuración sólo se aplica a `MergeTree` familia. Se puede especificar: - -- A nivel mundial en el [merge_tree](#server_configuration_parameters-merge_tree) sección de la `config.xml` file. - - ClickHouse utiliza la configuración para todas las tablas del servidor. Puede cambiar la configuración en cualquier momento. Las tablas existentes cambian su comportamiento cuando cambia la configuración. - -- Para cada tabla. - - Al crear una tabla, especifique la correspondiente [ajuste del motor](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table). El comportamiento de una tabla existente con esta configuración no cambia, incluso si la configuración global cambia. - -**Valores posibles** - -- 0 — Functionality is turned off. -- 1 — Functionality is turned on. - -Si `use_minimalistic_part_header_in_zookeeper = 1`, entonces [repetición](../../engines/table-engines/mergetree-family/replication.md) las tablas almacenan los encabezados de las partes de datos de forma compacta `znode`. Si la tabla contiene muchas columnas, este método de almacenamiento reduce significativamente el volumen de los datos almacenados en Zookeeper. - -!!! attention "Atención" - Después de aplicar `use_minimalistic_part_header_in_zookeeper = 1`, no puede degradar el servidor ClickHouse a una versión que no admite esta configuración. Tenga cuidado al actualizar ClickHouse en servidores de un clúster. No actualice todos los servidores a la vez. Es más seguro probar nuevas versiones de ClickHouse en un entorno de prueba o solo en unos pocos servidores de un clúster. - - Data part headers already stored with this setting can't be restored to their previous (non-compact) representation. - -**Valor predeterminado:** 0. - -## disable_internal_dns_cache {#server-settings-disable-internal-dns-cache} - -Deshabilita la memoria caché DNS interna. Recomendado para operar ClickHouse en sistemas -con infraestructura que cambia frecuentemente como Kubernetes. - -**Valor predeterminado:** 0. - -## dns_cache_update_period {#server-settings-dns-cache-update-period} - -El período de actualización de las direcciones IP almacenadas en la caché DNS interna de ClickHouse (en segundos). -La actualización se realiza de forma asíncrona, en un subproceso del sistema separado. - -**Valor predeterminado**: 15. - -## access_control_path {#access_control_path} - -Ruta de acceso a una carpeta donde un servidor ClickHouse almacena configuraciones de usuario y rol creadas por comandos SQL. - -Valor predeterminado: `/var/lib/clickhouse/access/`. - -**Ver también** - -- [Control de acceso y gestión de cuentas](../access-rights.md#access-control) - -[Artículo Original](https://clickhouse.tech/docs/en/operations/server_configuration_parameters/settings/) diff --git a/docs/es/operations/settings/constraints-on-settings.md b/docs/es/operations/settings/constraints-on-settings.md deleted file mode 100644 index fe385f6ddbb..00000000000 --- a/docs/es/operations/settings/constraints-on-settings.md +++ /dev/null @@ -1,75 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 62 -toc_title: "Restricciones en la configuraci\xF3n" ---- - -# Restricciones en la configuración {#constraints-on-settings} - -Las restricciones en los ajustes se pueden definir en el `profiles` sección de la `user.xml` el archivo de configuración y prohíba a los usuarios cambiar algunos de los ajustes `SET` consulta. -Las restricciones se definen como las siguientes: - -``` xml - - - - - lower_boundary - - - upper_boundary - - - lower_boundary - upper_boundary - - - - - - - -``` - -Si el usuario intenta violar las restricciones, se lanza una excepción y la configuración no se cambia. -Se admiten tres tipos de restricciones: `min`, `max`, `readonly`. El `min` y `max` Las restricciones especifican los límites superior e inferior para una configuración numérica y se pueden usar en combinación. El `readonly` constraint especifica que el usuario no puede cambiar la configuración correspondiente en absoluto. - -**Ejemplo:** Dejar `users.xml` incluye líneas: - -``` xml - - - 10000000000 - 0 - ... - - - 5000000000 - 20000000000 - - - - - - - -``` - -Las siguientes consultas arrojan excepciones: - -``` sql -SET max_memory_usage=20000000001; -SET max_memory_usage=4999999999; -SET force_index_by_date=1; -``` - -``` text -Code: 452, e.displayText() = DB::Exception: Setting max_memory_usage should not be greater than 20000000000. -Code: 452, e.displayText() = DB::Exception: Setting max_memory_usage should not be less than 5000000000. -Code: 452, e.displayText() = DB::Exception: Setting force_index_by_date should not be changed. -``` - -**Nota:** el `default` perfil tiene un manejo especial: todas las restricciones definidas para el `default` profile se convierten en las restricciones predeterminadas, por lo que restringen a todos los usuarios hasta que se anulan explícitamente para estos usuarios. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/constraints_on_settings/) diff --git a/docs/es/operations/settings/index.md b/docs/es/operations/settings/index.md deleted file mode 100644 index 37aab0a7e1b..00000000000 --- a/docs/es/operations/settings/index.md +++ /dev/null @@ -1,33 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Configuraci\xF3n" -toc_priority: 55 -toc_title: "Implantaci\xF3n" ---- - -# Configuración {#session-settings-intro} - -Hay varias maneras de realizar todos los ajustes descritos en esta sección de documentación. - -Los ajustes se configuran en capas, por lo que cada capa subsiguiente redefine los ajustes anteriores. - -Formas de configurar los ajustes, por orden de prioridad: - -- Ajustes en el `users.xml` archivo de configuración del servidor. - - Establecer en el elemento ``. - -- Configuración de la sesión. - - Enviar `SET setting=value` desde el cliente de consola ClickHouse en modo interactivo. - Del mismo modo, puede utilizar sesiones ClickHouse en el protocolo HTTP. Para hacer esto, debe especificar el `session_id` Parámetro HTTP. - -- Configuración de consulta. - - - Al iniciar el cliente de consola de ClickHouse en modo no interactivo, establezca el parámetro de inicio `--setting=value`. - - Al usar la API HTTP, pase los parámetros CGI (`URL?setting_1=value&setting_2=value...`). - -Los ajustes que solo se pueden realizar en el archivo de configuración del servidor no se tratan en esta sección. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/) diff --git a/docs/es/operations/settings/permissions-for-queries.md b/docs/es/operations/settings/permissions-for-queries.md deleted file mode 100644 index f9f669b876e..00000000000 --- a/docs/es/operations/settings/permissions-for-queries.md +++ /dev/null @@ -1,61 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: Permisos para consultas ---- - -# Permisos para consultas {#permissions_for_queries} - -Las consultas en ClickHouse se pueden dividir en varios tipos: - -1. Leer consultas de datos: `SELECT`, `SHOW`, `DESCRIBE`, `EXISTS`. -2. Escribir consultas de datos: `INSERT`, `OPTIMIZE`. -3. Cambiar la consulta de configuración: `SET`, `USE`. -4. [DDL](https://en.wikipedia.org/wiki/Data_definition_language) consulta: `CREATE`, `ALTER`, `RENAME`, `ATTACH`, `DETACH`, `DROP` `TRUNCATE`. -5. `KILL QUERY`. - -La siguiente configuración regula los permisos de usuario según el tipo de consulta: - -- [sólo lectura](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [Método de codificación de datos:](#settings_allow_ddl) — Restricts permissions for DDL queries. - -`KILL QUERY` se puede realizar con cualquier configuración. - -## sólo lectura {#settings_readonly} - -Restringe los permisos para leer datos, escribir datos y cambiar las consultas de configuración. - -Vea cómo las consultas se dividen en tipos [arriba](#permissions_for_queries). - -Valores posibles: - -- 0 — All queries are allowed. -- 1 — Only read data queries are allowed. -- 2 — Read data and change settings queries are allowed. - -Después de configurar `readonly = 1` el usuario no puede cambiar `readonly` y `allow_ddl` configuración en la sesión actual. - -Cuando se utiliza el `GET` método en el [Interfaz HTTP](../../interfaces/http.md), `readonly = 1` se establece automáticamente. Para modificar los datos, `POST` método. - -Configuración `readonly = 1` prohibir al usuario cambiar todas las configuraciones. Hay una manera de prohibir al usuario -de cambiar sólo ajustes específicos, para más detalles ver [restricciones en la configuración](constraints-on-settings.md). - -Valor predeterminado: 0 - -## Método de codificación de datos: {#settings_allow_ddl} - -Permite o niega [DDL](https://en.wikipedia.org/wiki/Data_definition_language) consulta. - -Vea cómo las consultas se dividen en tipos [arriba](#permissions_for_queries). - -Valores posibles: - -- 0 — DDL queries are not allowed. -- 1 — DDL queries are allowed. - -No se puede ejecutar `SET allow_ddl = 1` si `allow_ddl = 0` para la sesión actual. - -Valor predeterminado: 1 - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/permissions_for_queries/) diff --git a/docs/es/operations/settings/query-complexity.md b/docs/es/operations/settings/query-complexity.md deleted file mode 100644 index 82bc235c30d..00000000000 --- a/docs/es/operations/settings/query-complexity.md +++ /dev/null @@ -1,300 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 59 -toc_title: Restricciones en la complejidad de consultas ---- - -# Restricciones en la complejidad de consultas {#restrictions-on-query-complexity} - -Las restricciones en la complejidad de la consulta forman parte de la configuración. -Se utilizan para proporcionar una ejecución más segura desde la interfaz de usuario. -Casi todas las restricciones solo se aplican a `SELECT`. Para el procesamiento de consultas distribuidas, las restricciones se aplican en cada servidor por separado. - -ClickHouse comprueba las restricciones para las partes de datos, no para cada fila. Significa que puede exceder el valor de restricción con el tamaño de la parte de datos. - -Restricciones en el “maximum amount of something” puede tomar el valor 0, lo que significa “unrestricted”. -La mayoría de las restricciones también tienen un ‘overflow_mode’ establecer, lo que significa qué hacer cuando se excede el límite. -Puede tomar uno de dos valores: `throw` o `break`. Las restricciones en la agregación (group_by_overflow_mode) también tienen el valor `any`. - -`throw` – Throw an exception (default). - -`break` – Stop executing the query and return the partial result, as if the source data ran out. - -`any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. - -## Método de codificación de datos: {#settings_max_memory_usage} - -La cantidad máxima de RAM que se utiliza para ejecutar una consulta en un único servidor. - -En el archivo de configuración predeterminado, el máximo es de 10 GB. - -La configuración no tiene en cuenta el volumen de memoria disponible ni el volumen total de memoria en la máquina. -La restricción se aplica a una sola consulta dentro de un único servidor. -Usted puede utilizar `SHOW PROCESSLIST` para ver el consumo de memoria actual para cada consulta. -Además, el consumo máximo de memoria se rastrea para cada consulta y se escribe en el registro. - -El uso de memoria no se supervisa para los estados de ciertas funciones agregadas. - -El uso de memoria no se realiza un seguimiento completo de los estados de las funciones agregadas `min`, `max`, `any`, `anyLast`, `argMin`, `argMax` de `String` y `Array` argumento. - -El consumo de memoria también está restringido por los parámetros `max_memory_usage_for_user` y `max_memory_usage_for_all_queries`. - -## Max_memory_usage_for_user {#max-memory-usage-for-user} - -La cantidad máxima de RAM que se utilizará para ejecutar las consultas de un usuario en un único servidor. - -Los valores predeterminados se definen en [Configuración.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L288). De forma predeterminada, el importe no está restringido (`max_memory_usage_for_user = 0`). - -Ver también la descripción de [Método de codificación de datos:](#settings_max_memory_usage). - -## Todos los derechos reservados {#max-memory-usage-for-all-queries} - -La cantidad máxima de RAM que se utilizará para ejecutar todas las consultas en un único servidor. - -Los valores predeterminados se definen en [Configuración.h](https://github.com/ClickHouse/ClickHouse/blob/master/src/Core/Settings.h#L289). De forma predeterminada, el importe no está restringido (`max_memory_usage_for_all_queries = 0`). - -Ver también la descripción de [Método de codificación de datos:](#settings_max_memory_usage). - -## ¿Qué puedes encontrar en Neodigit {#max-rows-to-read} - -Las siguientes restricciones se pueden verificar en cada bloque (en lugar de en cada fila). Es decir, las restricciones se pueden romper un poco. - -Un número máximo de filas que se pueden leer de una tabla al ejecutar una consulta. - -## ¿Qué puedes encontrar en Neodigit {#max-bytes-to-read} - -Un número máximo de bytes (datos sin comprimir) que se pueden leer de una tabla al ejecutar una consulta. - -## Método de codificación de datos: {#read-overflow-mode} - -Qué hacer cuando el volumen de datos leídos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -## Método de codificación de datos: {#settings-max-rows-to-group-by} - -Un número máximo de claves únicas recibidas de la agregación. Esta configuración le permite limitar el consumo de memoria al agregar. - -## Grupo_by_overflow_mode {#group-by-overflow-mode} - -Qué hacer cuando el número de claves únicas para la agregación excede el límite: ‘throw’, ‘break’, o ‘any’. Por defecto, throw. -Uso de la ‘any’ valor le permite ejecutar una aproximación de GROUP BY. La calidad de esta aproximación depende de la naturaleza estadística de los datos. - -## max_bytes_before_external_group_by {#settings-max_bytes_before_external_group_by} - -Habilita o deshabilita la ejecución de `GROUP BY` en la memoria externa. Ver [GROUP BY en memoria externa](../../sql-reference/statements/select/group-by.md#select-group-by-in-external-memory). - -Valores posibles: - -- Volumen máximo de RAM (en bytes) que puede ser utilizado por el único [GROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-clause) operación. -- 0 — `GROUP BY` en la memoria externa deshabilitada. - -Valor predeterminado: 0. - -## Método de codificación de datos: {#max-rows-to-sort} - -Un número máximo de filas antes de ordenar. Esto le permite limitar el consumo de memoria al ordenar. - -## Método de codificación de datos: {#max-bytes-to-sort} - -Un número máximo de bytes antes de ordenar. - -## sort_overflow_mode {#sort-overflow-mode} - -Qué hacer si el número de filas recibidas antes de ordenar excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -## max_result_rows {#setting-max_result_rows} - -Límite en el número de filas en el resultado. También se comprueba si hay subconsultas y en servidores remotos cuando se ejecutan partes de una consulta distribuida. - -## max_result_bytes {#max-result-bytes} - -Límite en el número de bytes en el resultado. Lo mismo que el ajuste anterior. - -## result_overflow_mode {#result-overflow-mode} - -Qué hacer si el volumen del resultado excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -Utilizar ‘break’ es similar a usar LIMIT. `Break` interrumpe la ejecución sólo en el nivel de bloque. Esto significa que la cantidad de filas devueltas es mayor que [max_result_rows](#setting-max_result_rows), múltiplo de [max_block_size](settings.md#setting-max_block_size) y depende de [max_threads](settings.md#settings-max_threads). - -Ejemplo: - -``` sql -SET max_threads = 3, max_block_size = 3333; -SET max_result_rows = 3334, result_overflow_mode = 'break'; - -SELECT * -FROM numbers_mt(100000) -FORMAT Null; -``` - -Resultado: - -``` text -6666 rows in set. ... -``` - -## max_execution_time {#max-execution-time} - -Tiempo máximo de ejecución de la consulta en segundos. -En este momento, no se comprueba una de las etapas de clasificación, o al fusionar y finalizar funciones agregadas. - -## timeout_overflow_mode {#timeout-overflow-mode} - -Qué hacer si la consulta se ejecuta más de ‘max_execution_time’: ‘throw’ o ‘break’. Por defecto, throw. - -## Método de codificación de datos: {#min-execution-speed} - -Velocidad de ejecución mínima en filas por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. - -## Todos los derechos reservados {#min-execution-speed-bytes} - -Un número mínimo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es menor, se produce una excepción. - -## Max_execution_speed {#max-execution-speed} - -Un número máximo de filas de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. - -## Max_execution_speed_bytes {#max-execution-speed-bytes} - -Un número máximo de bytes de ejecución por segundo. Comprobado en cada bloque de datos cuando ‘timeout_before_checking_execution_speed’ expirar. Si la velocidad de ejecución es alta, la velocidad de ejecución se reducirá. - -## Tiempo de espera antes de comprobar_ejecución_velocidad {#timeout-before-checking-execution-speed} - -Comprueba que la velocidad de ejecución no sea demasiado lenta (no menos de ‘min_execution_speed’), después de que el tiempo especificado en segundos haya expirado. - -## Max_columns_to_read {#max-columns-to-read} - -Un número máximo de columnas que se pueden leer de una tabla en una sola consulta. Si una consulta requiere leer un mayor número de columnas, produce una excepción. - -## max_temporary_columns {#max-temporary-columns} - -Un número máximo de columnas temporales que se deben mantener en la memoria RAM al mismo tiempo cuando se ejecuta una consulta, incluidas las columnas constantes. Si hay más columnas temporales que esto, arroja una excepción. - -## max_temporary_non_const_columns {#max-temporary-non-const-columns} - -Lo mismo que ‘max_temporary_columns’, pero sin contar columnas constantes. -Tenga en cuenta que las columnas constantes se forman con bastante frecuencia cuando se ejecuta una consulta, pero requieren aproximadamente cero recursos informáticos. - -## max_subquery_depth {#max-subquery-depth} - -Profundidad máxima de anidamiento de subconsultas. Si las subconsultas son más profundas, se produce una excepción. De forma predeterminada, 100. - -## max_pipeline_depth {#max-pipeline-depth} - -Profundidad máxima de la tubería. Corresponde al número de transformaciones que realiza cada bloque de datos durante el procesamiento de consultas. Contado dentro de los límites de un único servidor. Si la profundidad de la canalización es mayor, se produce una excepción. Por defecto, 1000. - -## max_ast_depth {#max-ast-depth} - -Profundidad máxima de anidamiento de un árbol sintáctico de consulta. Si se supera, se produce una excepción. -En este momento, no se verifica durante el análisis, sino solo después de analizar la consulta. Es decir, se puede crear un árbol sintáctico demasiado profundo durante el análisis, pero la consulta fallará. Por defecto, 1000. - -## max_ast_elements {#max-ast-elements} - -Un número máximo de elementos en un árbol sintáctico de consulta. Si se supera, se produce una excepción. -De la misma manera que la configuración anterior, se verifica solo después de analizar la consulta. De forma predeterminada, 50.000. - -## Método de codificación de datos: {#max-rows-in-set} - -Un número máximo de filas para un conjunto de datos en la cláusula IN creada a partir de una subconsulta. - -## Método de codificación de datos: {#max-bytes-in-set} - -Número máximo de bytes (datos sin comprimir) utilizados por un conjunto en la cláusula IN creada a partir de una subconsulta. - -## set_overflow_mode {#set-overflow-mode} - -Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -## Método de codificación de datos: {#max-rows-in-distinct} - -Un número máximo de filas diferentes al usar DISTINCT. - -## Método de codificación de datos: {#max-bytes-in-distinct} - -Un número máximo de bytes utilizados por una tabla hash cuando se utiliza DISTINCT. - -## distinct_overflow_mode {#distinct-overflow-mode} - -Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -## max_rows_to_transfer {#max-rows-to-transfer} - -Un número máximo de filas que se pueden pasar a un servidor remoto o guardar en una tabla temporal cuando se utiliza GLOBAL IN. - -## max_bytes_to_transfer {#max-bytes-to-transfer} - -Un número máximo de bytes (datos sin comprimir) que se pueden pasar a un servidor remoto o guardar en una tabla temporal cuando se utiliza GLOBAL IN. - -## transfer_overflow_mode {#transfer-overflow-mode} - -Qué hacer cuando la cantidad de datos excede uno de los límites: ‘throw’ o ‘break’. Por defecto, throw. - -## Método de codificación de datos: {#settings-max_rows_in_join} - -Limita el número de filas de la tabla hash que se utiliza al unir tablas. - -Esta configuración se aplica a [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operaciones y la [Unir](../../engines/table-engines/special/join.md) motor de mesa. - -Si una consulta contiene varias combinaciones, ClickHouse comprueba esta configuración para cada resultado intermedio. - -ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilice el [join_overflow_mode](#settings-join_overflow_mode) configuración para elegir la acción. - -Valores posibles: - -- Entero positivo. -- 0 — Unlimited number of rows. - -Valor predeterminado: 0. - -## Método de codificación de datos: {#settings-max_bytes_in_join} - -Limita el tamaño en bytes de la tabla hash utilizada al unir tablas. - -Esta configuración se aplica a [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operaciones y [Unirse al motor de tabla](../../engines/table-engines/special/join.md). - -Si la consulta contiene combinaciones, ClickHouse comprueba esta configuración para cada resultado intermedio. - -ClickHouse puede proceder con diferentes acciones cuando se alcanza el límite. Utilizar [join_overflow_mode](#settings-join_overflow_mode) para elegir la acción. - -Valores posibles: - -- Entero positivo. -- 0 — Memory control is disabled. - -Valor predeterminado: 0. - -## join_overflow_mode {#settings-join_overflow_mode} - -Define qué acción realiza ClickHouse cuando se alcanza cualquiera de los siguientes límites de combinación: - -- [Método de codificación de datos:](#settings-max_bytes_in_join) -- [Método de codificación de datos:](#settings-max_rows_in_join) - -Valores posibles: - -- `THROW` — ClickHouse throws an exception and breaks operation. -- `BREAK` — ClickHouse breaks operation and doesn't throw an exception. - -Valor predeterminado: `THROW`. - -**Ver también** - -- [Cláusula JOIN](../../sql-reference/statements/select/join.md#select-join) -- [Unirse al motor de tabla](../../engines/table-engines/special/join.md) - -## max_partitions_per_insert_block {#max-partitions-per-insert-block} - -Limita el número máximo de particiones en un único bloque insertado. - -- Entero positivo. -- 0 — Unlimited number of partitions. - -Valor predeterminado: 100. - -**Detalles** - -Al insertar datos, ClickHouse calcula el número de particiones en el bloque insertado. Si el número de particiones es mayor que `max_partitions_per_insert_block`, ClickHouse lanza una excepción con el siguiente texto: - -> “Too many partitions for single INSERT block (more than” ¿Cómo puedo hacerlo? “). The limit is controlled by ‘max_partitions_per_insert_block’ setting. A large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).” - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/query_complexity/) diff --git a/docs/es/operations/settings/settings-profiles.md b/docs/es/operations/settings/settings-profiles.md deleted file mode 100644 index 3d96a2c8fba..00000000000 --- a/docs/es/operations/settings/settings-profiles.md +++ /dev/null @@ -1,81 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 61 -toc_title: "Perfiles de configuraci\xF3n" ---- - -# Perfiles de configuración {#settings-profiles} - -Un perfil de configuración es una colección de configuraciones agrupadas con el mismo nombre. - -!!! note "Información" - ClickHouse también es compatible [Flujo de trabajo controlado por SQL](../access-rights.md#access-control) para administrar perfiles de configuración. Recomendamos usarlo. - -Un perfil puede tener cualquier nombre. El perfil puede tener cualquier nombre. Puede especificar el mismo perfil para diferentes usuarios. Lo más importante que puede escribir en el perfil de configuración es `readonly=1`, que asegura el acceso de sólo lectura. - -Los perfiles de configuración pueden heredar unos de otros. Para usar la herencia, indique una o varias `profile` configuraciones antes de las demás configuraciones que se enumeran en el perfil. En caso de que se defina una configuración en diferentes perfiles, se utiliza la última definida. - -Para aplicar todos los ajustes de un perfil, establezca el `profile` configuración. - -Ejemplo: - -Instale el `web` perfil. - -``` sql -SET profile = 'web' -``` - -Los perfiles de configuración se declaran en el archivo de configuración del usuario. Esto suele ser `users.xml`. - -Ejemplo: - -``` xml - - - - - - 8 - - - - - 1000000000 - 100000000000 - - 1000000 - any - - 1000000 - 1000000000 - - 100000 - 100000000 - break - - 600 - 1000000 - 15 - - 25 - 100 - 50 - - 2 - 25 - 50 - 100 - - 1 - - -``` - -El ejemplo especifica dos perfiles: `default` y `web`. - -El `default` tiene un propósito especial: siempre debe estar presente y se aplica al iniciar el servidor. En otras palabras, el `default` perfil contiene la configuración predeterminada. - -El `web` profile es un perfil regular que se puede establecer utilizando el `SET` consulta o utilizando un parámetro URL en una consulta HTTP. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/settings_profiles/) diff --git a/docs/es/operations/settings/settings-users.md b/docs/es/operations/settings/settings-users.md deleted file mode 100644 index 1c1ac7914f0..00000000000 --- a/docs/es/operations/settings/settings-users.md +++ /dev/null @@ -1,164 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 63 -toc_title: "Configuraci\xF3n del usuario" ---- - -# Configuración del usuario {#user-settings} - -El `users` sección de la `user.xml` el archivo de configuración contiene la configuración del usuario. - -!!! note "Información" - ClickHouse también es compatible [Flujo de trabajo controlado por SQL](../access-rights.md#access-control) para la gestión de usuarios. Recomendamos usarlo. - -Estructura del `users` apartado: - -``` xml - - - - - - - - 0|1 - - - - - profile_name - - default - - - - - expression - - - - - - -``` - -### user_name/contraseña {#user-namepassword} - -La contraseña se puede especificar en texto sin formato o en SHA256 (formato hexagonal). - -- Para asignar una contraseña en texto sin formato (**no se recomienda**), colóquelo en un `password` elemento. - - Por ejemplo, `qwerty`. La contraseña se puede dejar en blanco. - - - -- Para asignar una contraseña utilizando su hash SHA256, colóquela en un `password_sha256_hex` elemento. - - Por ejemplo, `65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5`. - - Ejemplo de cómo generar una contraseña desde el shell: - - PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-' - - La primera línea del resultado es la contraseña. La segunda línea es el hash SHA256 correspondiente. - - - -- Para la compatibilidad con los clientes MySQL, la contraseña se puede especificar en doble hash SHA1. Colóquelo en `password_double_sha1_hex` elemento. - - Por ejemplo, `08b4a0f1de6ad37da17359e592c8d74788a83eb0`. - - Ejemplo de cómo generar una contraseña desde el shell: - - PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha1sum | tr -d '-' | xxd -r -p | sha1sum | tr -d '-' - - La primera línea del resultado es la contraseña. La segunda línea es el hash SHA1 doble correspondiente. - -### access_management {#access_management-user-setting} - -Esta configuración habilita deshabilita el uso de [control de acceso y gestión de cuentas](../access-rights.md#access-control) para el usuario. - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -### user_name/redes {#user-namenetworks} - -Lista de redes desde las que el usuario puede conectarse al servidor ClickHouse. - -Cada elemento de la lista puede tener una de las siguientes formas: - -- `` — IP address or network mask. - - Ejemplos: `213.180.204.3`, `10.0.0.1/8`, `10.0.0.1/255.255.255.0`, `2a02:6b8::3`, `2a02:6b8::3/64`, `2a02:6b8::3/ffff:ffff:ffff:ffff::`. - -- `` — Hostname. - - Ejemplo: `example01.host.ru`. - - Para comprobar el acceso, se realiza una consulta DNS y todas las direcciones IP devueltas se comparan con la dirección del mismo nivel. - -- `` — Regular expression for hostnames. - - Ejemplo, `^example\d\d-\d\d-\d\.host\.ru$` - - Para comprobar el acceso, un [Consulta de DNS PTR](https://en.wikipedia.org/wiki/Reverse_DNS_lookup) se realiza para la dirección del mismo nivel y luego se aplica la expresión regular especificada. A continuación, se realiza otra consulta DNS para los resultados de la consulta PTR y todas las direcciones recibidas se comparan con la dirección del mismo nivel. Recomendamos encarecidamente que regexp termine con $ . - -Todos los resultados de las solicitudes DNS se almacenan en caché hasta que el servidor se reinicia. - -**Ejemplos** - -Para abrir el acceso del usuario desde cualquier red, especifique: - -``` xml -::/0 -``` - -!!! warning "Advertencia" - No es seguro abrir el acceso desde cualquier red a menos que tenga un firewall configurado correctamente o el servidor no esté conectado directamente a Internet. - -Para abrir el acceso solo desde localhost, especifique: - -``` xml -::1 -127.0.0.1 -``` - -### user_name/perfil {#user-nameprofile} - -Puede asignar un perfil de configuración para el usuario. Los perfiles de configuración se configuran en una sección separada del `users.xml` file. Para obtener más información, consulte [Perfiles de configuración](settings-profiles.md). - -### user_name/cuota {#user-namequota} - -Las cuotas le permiten realizar un seguimiento o limitar el uso de recursos durante un período de tiempo. Las cuotas se configuran en el `quotas` -sección de la `users.xml` archivo de configuración. - -Puede asignar un conjunto de cuotas para el usuario. Para obtener una descripción detallada de la configuración de las cuotas, consulte [Cuota](../quotas.md#quotas). - -### nombre_usuario/bases de datos {#user-namedatabases} - -En esta sección, puede limitar las filas devueltas por ClickHouse para `SELECT` consultas realizadas por el usuario actual, implementando así la seguridad básica a nivel de fila. - -**Ejemplo** - -La siguiente configuración obliga a que el usuario `user1` sólo puede ver las filas de `table1` como resultado de `SELECT` consultas, donde el valor de la `id` campo es 1000. - -``` xml - - - - - id = 1000 - - - - -``` - -El `filter` puede ser cualquier expresión que resulte en un [UInt8](../../sql-reference/data-types/int-uint.md)-tipo de valor. Por lo general, contiene comparaciones y operadores lógicos. Filas de `database_name.table1` donde los resultados del filtro a 0 no se devuelven para este usuario. El filtrado es incompatible con `PREWHERE` operaciones y desactiva `WHERE→PREWHERE` optimización. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/settings_users/) diff --git a/docs/es/operations/settings/settings.md b/docs/es/operations/settings/settings.md deleted file mode 100644 index 62511dd9fc0..00000000000 --- a/docs/es/operations/settings/settings.md +++ /dev/null @@ -1,1254 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Configuración {#settings} - -## distributed_product_mode {#distributed-product-mode} - -Cambia el comportamiento de [subconsultas distribuidas](../../sql-reference/operators/in.md). - -ClickHouse applies this setting when the query contains the product of distributed tables, i.e. when the query for a distributed table contains a non-GLOBAL subquery for the distributed table. - -Restricción: - -- Solo se aplica para las subconsultas IN y JOIN. -- Solo si la sección FROM utiliza una tabla distribuida que contiene más de un fragmento. -- Si la subconsulta se refiere a una tabla distribuida que contiene más de un fragmento. -- No se usa para un valor de tabla [remoto](../../sql-reference/table-functions/remote.md) función. - -Valores posibles: - -- `deny` — Default value. Prohibits using these types of subqueries (returns the “Double-distributed in/JOIN subqueries is denied” salvedad). -- `local` — Replaces the database and table in the subquery with local ones for the destination server (shard), leaving the normal `IN`/`JOIN.` -- `global` — Replaces the `IN`/`JOIN` consulta con `GLOBAL IN`/`GLOBAL JOIN.` -- `allow` — Allows the use of these types of subqueries. - -## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} - -Activa el pushdown de predicado en `SELECT` consulta. - -La extracción de predicados puede reducir significativamente el tráfico de red para consultas distribuidas. - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 1. - -Uso - -Considere las siguientes consultas: - -1. `SELECT count() FROM test_table WHERE date = '2018-10-10'` -2. `SELECT count() FROM (SELECT * FROM test_table) WHERE date = '2018-10-10'` - -Si `enable_optimize_predicate_expression = 1`, entonces el tiempo de ejecución de estas consultas es igual porque se aplica ClickHouse `WHERE` a la subconsulta al procesarla. - -Si `enable_optimize_predicate_expression = 0`, entonces el tiempo de ejecución de la segunda consulta es mucho más largo, porque el `WHERE` cláusula se aplica a todos los datos después de que finalice la subconsulta. - -## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} - -Fuerza una consulta a una réplica obsoleta si los datos actualizados no están disponibles. Ver [Replicación](../../engines/table-engines/mergetree-family/replication.md). - -ClickHouse selecciona la más relevante de las réplicas obsoletas de la tabla. - -Se utiliza al realizar `SELECT` desde una tabla distribuida que apunta a tablas replicadas. - -De forma predeterminada, 1 (habilitado). - -## Fecha de nacimiento {#settings-force_index_by_date} - -Deshabilita la ejecución de consultas si el índice no se puede usar por fecha. - -Funciona con tablas de la familia MergeTree. - -Si `force_index_by_date=1`, ClickHouse comprueba si la consulta tiene una condición de clave de fecha que se puede usar para restringir intervalos de datos. Si no hay una condición adecuada, arroja una excepción. Sin embargo, no comprueba si la condición reduce la cantidad de datos a leer. Por ejemplo, la condición `Date != ' 2000-01-01 '` es aceptable incluso cuando coincide con todos los datos de la tabla (es decir, ejecutar la consulta requiere un escaneo completo). Para obtener más información acerca de los intervalos de datos en las tablas MergeTree, vea [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). - -## force_primary_key {#force-primary-key} - -Deshabilita la ejecución de consultas si no es posible la indexación mediante la clave principal. - -Funciona con tablas de la familia MergeTree. - -Si `force_primary_key=1`, ClickHouse comprueba si la consulta tiene una condición de clave principal que se puede usar para restringir rangos de datos. Si no hay una condición adecuada, arroja una excepción. Sin embargo, no comprueba si la condición reduce la cantidad de datos a leer. Para obtener más información acerca de los intervalos de datos en las tablas MergeTree, consulte [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md). - -## Formato_esquema {#format-schema} - -Este parámetro es útil cuando se utilizan formatos que requieren una definición de esquema, como [Cap'n Proto](https://capnproto.org/) o [Protobuf](https://developers.google.com/protocol-buffers/). El valor depende del formato. - -## fsync_metadata {#fsync-metadata} - -Habilita o deshabilita [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) al escribir `.sql` file. Habilitado de forma predeterminada. - -Tiene sentido desactivarlo si el servidor tiene millones de pequeñas tablas que se crean y destruyen constantemente. - -## enable_http_compression {#settings-enable_http_compression} - -Habilita o deshabilita la compresión de datos en la respuesta a una solicitud HTTP. - -Para obtener más información, lea el [Descripción de la interfaz HTTP](../../interfaces/http.md). - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -## http_zlib_compression_level {#settings-http_zlib_compression_level} - -Establece el nivel de compresión de datos en la respuesta a una solicitud HTTP si [enable_http_compression = 1](#settings-enable_http_compression). - -Valores posibles: Números del 1 al 9. - -Valor predeterminado: 3. - -## http_native_compression_disable_checksumming_on_decompress {#settings-http_native_compression_disable_checksumming_on_decompress} - -Habilita o deshabilita la verificación de suma de comprobación al descomprimir los datos HTTP POST del cliente. Se usa solo para el formato de compresión nativa ClickHouse (no se usa con `gzip` o `deflate`). - -Para obtener más información, lea el [Descripción de la interfaz HTTP](../../interfaces/http.md). - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -## send_progress_in_http_headers {#settings-send_progress_in_http_headers} - -Habilita o deshabilita `X-ClickHouse-Progress` Encabezados de respuesta HTTP en `clickhouse-server` respuesta. - -Para obtener más información, lea el [Descripción de la interfaz HTTP](../../interfaces/http.md). - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -## Nombre de la red inalámbrica (SSID): {#setting-max_http_get_redirects} - -Limita el número máximo de saltos de redirección HTTP GET para [URL](../../engines/table-engines/special/url.md)-mesas de motor. La configuración se aplica a ambos tipos de tablas: las creadas por [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) consulta y por el [URL](../../sql-reference/table-functions/url.md) función de la tabla. - -Valores posibles: - -- Cualquier número entero positivo de saltos. -- 0 — No hops allowed. - -Valor predeterminado: 0. - -## Entrada_format_allow_errors_num {#settings-input_format_allow_errors_num} - -Establece el número máximo de errores aceptables al leer desde formatos de texto (CSV, TSV, etc.). - -El valor predeterminado es 0. - -Siempre emparejarlo con `input_format_allow_errors_ratio`. - -Si se produjo un error al leer filas, pero el contador de errores sigue siendo menor que `input_format_allow_errors_num`, ClickHouse ignora la fila y pasa a la siguiente. - -Si ambos `input_format_allow_errors_num` y `input_format_allow_errors_ratio` se exceden, ClickHouse lanza una excepción. - -## Entrada_format_allow_errors_ratio {#settings-input_format_allow_errors_ratio} - -Establece el porcentaje máximo de errores permitidos al leer desde formatos de texto (CSV, TSV, etc.). -El porcentaje de errores se establece como un número de punto flotante entre 0 y 1. - -El valor predeterminado es 0. - -Siempre emparejarlo con `input_format_allow_errors_num`. - -Si se produjo un error al leer filas, pero el contador de errores sigue siendo menor que `input_format_allow_errors_ratio`, ClickHouse ignora la fila y pasa a la siguiente. - -Si ambos `input_format_allow_errors_num` y `input_format_allow_errors_ratio` se exceden, ClickHouse lanza una excepción. - -## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} - -Habilita o deshabilita el analizador SQL completo si el analizador de secuencias rápidas no puede analizar los datos. Esta configuración sólo se utiliza para [Valor](../../interfaces/formats.md#data-format-values) formato en la inserción de datos. Para obtener más información sobre el análisis de sintaxis, consulte [Sintaxis](../../sql-reference/syntax.md) apartado. - -Valores posibles: - -- 0 — Disabled. - - En este caso, debe proporcionar datos con formato. Ver el [Formato](../../interfaces/formats.md) apartado. - -- 1 — Enabled. - - En este caso, puede usar una expresión SQL como valor, pero la inserción de datos es mucho más lenta de esta manera. Si inserta solo datos con formato, ClickHouse se comporta como si el valor de configuración fuera 0. - -Valor predeterminado: 1. - -Ejemplo de uso - -Inserte el [FechaHora](../../sql-reference/data-types/datetime.md) valor de tipo con los diferentes ajustes. - -``` sql -SET input_format_values_interpret_expressions = 0; -INSERT INTO datetime_t VALUES (now()) -``` - -``` text -Exception on client: -Code: 27. DB::Exception: Cannot parse input: expected ) before: now()): (at row 1) -``` - -``` sql -SET input_format_values_interpret_expressions = 1; -INSERT INTO datetime_t VALUES (now()) -``` - -``` text -Ok. -``` - -La última consulta es equivalente a la siguiente: - -``` sql -SET input_format_values_interpret_expressions = 0; -INSERT INTO datetime_t SELECT now() -``` - -``` text -Ok. -``` - -## input_format_values_deduce_templates_of_expressions {#settings-input_format_values_deduce_templates_of_expressions} - -Habilita o deshabilita la deducción de plantilla para expresiones SQL en [Valor](../../interfaces/formats.md#data-format-values) formato. Permite analizar e interpretar expresiones en `Values` mucho más rápido si las expresiones en filas consecutivas tienen la misma estructura. ClickHouse intenta deducir la plantilla de una expresión, analizar las siguientes filas utilizando esta plantilla y evaluar la expresión en un lote de filas analizadas correctamente. - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 1. - -Para la siguiente consulta: - -``` sql -INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (upper('Values')), ... -``` - -- Si `input_format_values_interpret_expressions=1` y `format_values_deduce_templates_of_expressions=0`, las expresiones se interpretan por separado para cada fila (esto es muy lento para un gran número de filas). -- Si `input_format_values_interpret_expressions=0` y `format_values_deduce_templates_of_expressions=1`, las expresiones en la primera, segunda y tercera filas se analizan usando la plantilla `lower(String)` e interpretados juntos, la expresión en la cuarta fila se analiza con otra plantilla (`upper(String)`). -- Si `input_format_values_interpret_expressions=1` y `format_values_deduce_templates_of_expressions=1`, lo mismo que en el caso anterior, pero también permite la alternativa a la interpretación de expresiones por separado si no es posible deducir la plantilla. - -## Entrada_format_values_accurate_types_of_literals {#settings-input-format-values-accurate-types-of-literals} - -Esta configuración sólo se utiliza cuando `input_format_values_deduce_templates_of_expressions = 1`. Puede suceder que las expresiones para alguna columna tengan la misma estructura, pero contengan literales numéricos de diferentes tipos, por ejemplo - -``` sql -(..., abs(0), ...), -- UInt64 literal -(..., abs(3.141592654), ...), -- Float64 literal -(..., abs(-1), ...), -- Int64 literal -``` - -Valores posibles: - -- 0 — Disabled. - - In this case, ClickHouse may use a more general type for some literals (e.g., `Float64` o `Int64` en lugar de `UInt64` para `42`), pero puede causar problemas de desbordamiento y precisión. - -- 1 — Enabled. - - En este caso, ClickHouse comprueba el tipo real de literal y utiliza una plantilla de expresión del tipo correspondiente. En algunos casos, puede ralentizar significativamente la evaluación de expresiones en `Values`. - -Valor predeterminado: 1. - -## Entrada_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} - -Al realizar `INSERT` consultas, reemplace los valores de columna de entrada omitidos con valores predeterminados de las columnas respectivas. Esta opción sólo se aplica a [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) y [TabSeparated](../../interfaces/formats.md#tabseparated) formato. - -!!! note "Nota" - Cuando esta opción está habilitada, los metadatos de la tabla extendida se envían del servidor al cliente. Consume recursos informáticos adicionales en el servidor y puede reducir el rendimiento. - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 1. - -## input_format_tsv_empty_as_default {#settings-input-format-tsv-empty-as-default} - -Cuando esté habilitado, reemplace los campos de entrada vacíos en TSV con valores predeterminados. Para expresiones predeterminadas complejas `input_format_defaults_for_omitted_fields` debe estar habilitado también. - -Deshabilitado de forma predeterminada. - -## input_format_null_as_default {#settings-input-format-null-as-default} - -Habilita o deshabilita el uso de valores predeterminados si los datos de entrada `NULL`, pero el tipo de datos de la columna correspondiente en no `Nullable(T)` (para formatos de entrada de texto). - -## input_format_skip_unknown_fields {#settings-input-format-skip-unknown-fields} - -Habilita o deshabilita omitir la inserción de datos adicionales. - -Al escribir datos, ClickHouse produce una excepción si los datos de entrada contienen columnas que no existen en la tabla de destino. Si la omisión está habilitada, ClickHouse no inserta datos adicionales y no lanza una excepción. - -Formatos soportados: - -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) -- [TSKV](../../interfaces/formats.md#tskv) - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -## Entrada_format_import_nested_json {#settings-input_format_import_nested_json} - -Habilita o deshabilita la inserción de datos JSON con objetos anidados. - -Formatos soportados: - -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -Ver también: - -- [Uso de estructuras anidadas](../../interfaces/formats.md#jsoneachrow-nested) con el `JSONEachRow` formato. - -## Entrada_format_with_names_use_header {#settings-input-format-with-names-use-header} - -Habilita o deshabilita la comprobación del orden de las columnas al insertar datos. - -Para mejorar el rendimiento de la inserción, se recomienda deshabilitar esta comprobación si está seguro de que el orden de columna de los datos de entrada es el mismo que en la tabla de destino. - -Formatos soportados: - -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 1. - -## Date_time_input_format {#settings-date_time_input_format} - -Permite elegir un analizador de la representación de texto de fecha y hora. - -La configuración no se aplica a [Funciones de fecha y hora](../../sql-reference/functions/date-time-functions.md). - -Valores posibles: - -- `'best_effort'` — Enables extended parsing. - - ClickHouse puede analizar el básico `YYYY-MM-DD HH:MM:SS` formato y todo [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) formatos de fecha y hora. Por ejemplo, `'2018-06-08T01:02:03.000Z'`. - -- `'basic'` — Use basic parser. - - ClickHouse puede analizar solo lo básico `YYYY-MM-DD HH:MM:SS` formato. Por ejemplo, `'2019-08-20 10:18:56'`. - -Valor predeterminado: `'basic'`. - -Ver también: - -- [Tipo de datos DateTime.](../../sql-reference/data-types/datetime.md) -- [Funciones para trabajar con fechas y horas.](../../sql-reference/functions/date-time-functions.md) - -## Por favor, introduzca su dirección de correo electrónico {#settings-join_default_strictness} - -Establece el rigor predeterminado para [Cláusulas JOIN](../../sql-reference/statements/select/join.md#select-join). - -Valores posibles: - -- `ALL` — If the right table has several matching rows, ClickHouse creates a [Producto cartesiano](https://en.wikipedia.org/wiki/Cartesian_product) de filas coincidentes. Esta es la normal `JOIN` comportamiento de SQL estándar. -- `ANY` — If the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` y `ALL` son los mismos. -- `ASOF` — For joining sequences with an uncertain match. -- `Empty string` — If `ALL` o `ANY` no se especifica en la consulta, ClickHouse produce una excepción. - -Valor predeterminado: `ALL`. - -## join_any_take_last_row {#settings-join_any_take_last_row} - -Cambia el comportamiento de las operaciones de unión con `ANY` rigor. - -!!! warning "Atención" - Esta configuración sólo se aplica a `JOIN` operaciones con [Unir](../../engines/table-engines/special/join.md) mesas de motores. - -Valores posibles: - -- 0 — If the right table has more than one matching row, only the first one found is joined. -- 1 — If the right table has more than one matching row, only the last one found is joined. - -Valor predeterminado: 0. - -Ver también: - -- [Cláusula JOIN](../../sql-reference/statements/select/join.md#select-join) -- [Unirse al motor de tabla](../../engines/table-engines/special/join.md) -- [Por favor, introduzca su dirección de correo electrónico](#settings-join_default_strictness) - -## Sistema abierto {#join_use_nulls} - -Establece el tipo de [JOIN](../../sql-reference/statements/select/join.md) comportamiento. Al fusionar tablas, pueden aparecer celdas vacías. ClickHouse los rellena de manera diferente según esta configuración. - -Valores posibles: - -- 0 — The empty cells are filled with the default value of the corresponding field type. -- 1 — `JOIN` se comporta de la misma manera que en SQL estándar. El tipo del campo correspondiente se convierte en [NULL](../../sql-reference/data-types/nullable.md#data_type-nullable), y las celdas vacías se llenan con [NULL](../../sql-reference/syntax.md). - -Valor predeterminado: 0. - -## max_block_size {#setting-max_block_size} - -En ClickHouse, los datos se procesan mediante bloques (conjuntos de partes de columna). Los ciclos de procesamiento interno para un solo bloque son lo suficientemente eficientes, pero hay gastos notables en cada bloque. El `max_block_size` set es una recomendación para el tamaño del bloque (en un recuento de filas) para cargar desde las tablas. El tamaño del bloque no debe ser demasiado pequeño, por lo que los gastos en cada bloque aún se notan, pero no demasiado grande para que la consulta con LIMIT que se complete después del primer bloque se procese rápidamente. El objetivo es evitar consumir demasiada memoria al extraer un gran número de columnas en múltiples subprocesos y preservar al menos alguna localidad de caché. - -Valor predeterminado: 65,536. - -Bloquea el tamaño de `max_block_size` no siempre se cargan desde la tabla. Si es obvio que se deben recuperar menos datos, se procesa un bloque más pequeño. - -## preferred_block_size_bytes {#preferred-block-size-bytes} - -Utilizado para el mismo propósito que `max_block_size`, pero establece el tamaño de bloque recomendado en bytes adaptándolo al número de filas en el bloque. -Sin embargo, el tamaño del bloque no puede ser más que `max_block_size` filas. -Por defecto: 1,000,000. Solo funciona cuando se lee desde los motores MergeTree. - -## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} - -Si el número de filas que se leerán de un fichero [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md) mesa excede `merge_tree_min_rows_for_concurrent_read` luego ClickHouse intenta realizar una lectura simultánea de este archivo en varios hilos. - -Valores posibles: - -- Cualquier entero positivo. - -Valor predeterminado: 163840. - -## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read} - -Si el número de bytes a leer de un archivo de un [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)-La tabla del motor excede `merge_tree_min_bytes_for_concurrent_read`, entonces ClickHouse intenta leer simultáneamente este archivo en varios subprocesos. - -Valor posible: - -- Cualquier entero positivo. - -Valor predeterminado: 251658240. - -## Método de codificación de datos: {#setting-merge-tree-min-rows-for-seek} - -Si la distancia entre dos bloques de datos que se leen en un archivo es menor que `merge_tree_min_rows_for_seek` filas, luego ClickHouse no busca a través del archivo, sino que lee los datos secuencialmente. - -Valores posibles: - -- Cualquier entero positivo. - -Valor predeterminado: 0. - -## merge_tree_min_bytes_for_seek {#setting-merge-tree-min-bytes-for-seek} - -Si la distancia entre dos bloques de datos que se leen en un archivo es menor que `merge_tree_min_bytes_for_seek` bytes, luego ClickHouse lee secuencialmente un rango de archivos que contiene ambos bloques, evitando así la búsqueda adicional. - -Valores posibles: - -- Cualquier entero positivo. - -Valor predeterminado: 0. - -## merge_tree_coarse_index_granularity {#setting-merge-tree-coarse-index-granularity} - -Al buscar datos, ClickHouse comprueba las marcas de datos en el archivo de índice. Si ClickHouse encuentra que las claves requeridas están en algún rango, divide este rango en `merge_tree_coarse_index_granularity` subintervalos y busca las claves necesarias allí de forma recursiva. - -Valores posibles: - -- Cualquier entero incluso positivo. - -Valor predeterminado: 8. - -## merge_tree_max_rows_to_use_cache {#setting-merge-tree-max-rows-to-use-cache} - -Si ClickHouse debería leer más de `merge_tree_max_rows_to_use_cache` en una consulta, no usa la memoria caché de bloques sin comprimir. - -La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. - -Valores posibles: - -- Cualquier entero positivo. - -Default value: 128 ✕ 8192. - -## merge_tree_max_bytes_to_use_cache {#setting-merge-tree-max-bytes-to-use-cache} - -Si ClickHouse debería leer más de `merge_tree_max_bytes_to_use_cache` bytes en una consulta, no usa el caché de bloques sin comprimir. - -La memoria caché de bloques sin comprimir almacena datos extraídos para consultas. ClickHouse utiliza esta memoria caché para acelerar las respuestas a pequeñas consultas repetidas. Esta configuración protege la memoria caché del deterioro de las consultas que leen una gran cantidad de datos. El [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuración del servidor define el tamaño de la memoria caché de bloques sin comprimir. - -Valor posible: - -- Cualquier entero positivo. - -Valor predeterminado: 2013265920. - -## Todos los derechos reservados {#settings-min-bytes-to-use-direct-io} - -El volumen de datos mínimo necesario para utilizar el acceso directo de E/S al disco de almacenamiento. - -ClickHouse usa esta configuración al leer datos de tablas. Si el volumen total de almacenamiento de todos los datos a leer excede `min_bytes_to_use_direct_io` luego ClickHouse lee los datos del disco de almacenamiento con el `O_DIRECT` opcion. - -Valores posibles: - -- 0 — Direct I/O is disabled. -- Entero positivo. - -Valor predeterminado: 0. - -## Log_queries {#settings-log-queries} - -Configuración del registro de consultas. - -Las consultas enviadas a ClickHouse con esta configuración se registran de acuerdo con las reglas [query_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query-log) parámetro de configuración del servidor. - -Ejemplo: - -``` text -log_queries=1 -``` - -## Nombre de la red inalámbrica (SSID): {#settings-log-queries-min-type} - -`query_log` tipo mínimo para iniciar sesión. - -Valores posibles: -- `QUERY_START` (`=1`) -- `QUERY_FINISH` (`=2`) -- `EXCEPTION_BEFORE_START` (`=3`) -- `EXCEPTION_WHILE_PROCESSING` (`=4`) - -Valor predeterminado: `QUERY_START`. - -Se puede usar para limitar a qué entiries va `query_log`, digamos que eres interesante solo en errores, entonces puedes usar `EXCEPTION_WHILE_PROCESSING`: - -``` text -log_queries_min_type='EXCEPTION_WHILE_PROCESSING' -``` - -## Log_query_threads {#settings-log-query-threads} - -Configuración del registro de subprocesos de consulta. - -Los subprocesos de consultas ejecutados por ClickHouse con esta configuración se registran de acuerdo con las reglas en el [Sistema abierto.](../server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) parámetro de configuración del servidor. - -Ejemplo: - -``` text -log_query_threads=1 -``` - -## Max_insert_block_size {#settings-max_insert_block_size} - -El tamaño de los bloques a formar para su inserción en una tabla. -Esta configuración solo se aplica en los casos en que el servidor forma los bloques. -Por ejemplo, para un INSERT a través de la interfaz HTTP, el servidor analiza el formato de datos y forma bloques del tamaño especificado. -Pero al usar clickhouse-client, el cliente analiza los datos en sí, y el ‘max_insert_block_size’ configuración en el servidor no afecta el tamaño de los bloques insertados. -La configuración tampoco tiene un propósito cuando se usa INSERT SELECT , ya que los datos se insertan usando los mismos bloques que se forman después de SELECT . - -Valor predeterminado: 1.048.576. - -El valor predeterminado es ligeramente más que `max_block_size`. La razón de esto se debe a que ciertos motores de mesa (`*MergeTree`) formar una parte de datos en el disco para cada bloque insertado, que es una entidad bastante grande. Similar, `*MergeTree` las tablas ordenan los datos durante la inserción y un tamaño de bloque lo suficientemente grande permiten clasificar más datos en la RAM. - -## Nombre de la red inalámbrica (SSID): {#min-insert-block-size-rows} - -Establece el número mínimo de filas en el bloque que se pueden insertar en una tabla `INSERT` consulta. Los bloques de menor tamaño se aplastan en otros más grandes. - -Valores posibles: - -- Entero positivo. -- 0 — Squashing disabled. - -Valor predeterminado: 1048576. - -## Todos los derechos reservados {#min-insert-block-size-bytes} - -Establece el número mínimo de bytes en el bloque que se pueden insertar en una tabla `INSERT` consulta. Los bloques de menor tamaño se aplastan en otros más grandes. - -Valores posibles: - -- Entero positivo. -- 0 — Squashing disabled. - -Valor predeterminado: 268435456. - -## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} - -Deshabilita las réplicas rezagadas para consultas distribuidas. Ver [Replicación](../../engines/table-engines/mergetree-family/replication.md). - -Establece el tiempo en segundos. Si una réplica tiene un retraso superior al valor establecido, no se utiliza esta réplica. - -Valor predeterminado: 300. - -Se utiliza al realizar `SELECT` desde una tabla distribuida que apunta a tablas replicadas. - -## max_threads {#settings-max_threads} - -El número máximo de subprocesos de procesamiento de consultas, excluyendo subprocesos para recuperar datos de servidores ‘max_distributed_connections’ parámetro). - -Este parámetro se aplica a los subprocesos que realizan las mismas etapas de la canalización de procesamiento de consultas en paralelo. -Por ejemplo, al leer desde una tabla, si es posible evaluar expresiones con funciones, filtre con WHERE y preagregue para GROUP BY en paralelo usando al menos ‘max_threads’ número de hilos, entonces ‘max_threads’ se utilizan. - -Valor predeterminado: el número de núcleos de CPU físicos. - -Si normalmente se ejecuta menos de una consulta SELECT en un servidor a la vez, establezca este parámetro en un valor ligeramente inferior al número real de núcleos de procesador. - -Para las consultas que se completan rápidamente debido a un LIMIT, puede establecer un ‘max_threads’. Por ejemplo, si el número necesario de entradas se encuentra en cada bloque y max_threads = 8, entonces se recuperan 8 bloques, aunque hubiera sido suficiente leer solo uno. - -Cuanto menor sea el `max_threads` valor, menos memoria se consume. - -## Método de codificación de datos: {#settings-max-insert-threads} - -El número máximo de subprocesos para ejecutar el `INSERT SELECT` consulta. - -Valores posibles: - -- 0 (or 1) — `INSERT SELECT` sin ejecución paralela. -- Entero positivo. Más grande que 1. - -Valor predeterminado: 0. - -Paralelo `INSERT SELECT` sólo tiene efecto si el `SELECT` parte se ejecuta en paralelo, ver [max_threads](#settings-max_threads) configuración. -Los valores más altos conducirán a un mayor uso de memoria. - -## max_compress_block_size {#max-compress-block-size} - -El tamaño máximo de bloques de datos sin comprimir antes de comprimir para escribir en una tabla. De forma predeterminada, 1.048.576 (1 MiB). Si se reduce el tamaño, la tasa de compresión se reduce significativamente, la velocidad de compresión y descompresión aumenta ligeramente debido a la localidad de la memoria caché, y se reduce el consumo de memoria. Por lo general, no hay ninguna razón para cambiar esta configuración. - -No confunda bloques para la compresión (un fragmento de memoria que consta de bytes) con bloques para el procesamiento de consultas (un conjunto de filas de una tabla). - -## Descripción del producto {#min-compress-block-size} - -Para [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md)" tabla. Para reducir la latencia al procesar consultas, un bloque se comprime al escribir la siguiente marca si su tamaño es al menos ‘min_compress_block_size’. De forma predeterminada, 65.536. - -El tamaño real del bloque, si los datos sin comprimir son menores que ‘max_compress_block_size’, no es menor que este valor y no menor que el volumen de datos para una marca. - -Veamos un ejemplo. Supongamos que ‘index_granularity’ se estableció en 8192 durante la creación de la tabla. - -Estamos escribiendo una columna de tipo UInt32 (4 bytes por valor). Al escribir 8192 filas, el total será de 32 KB de datos. Como min_compress_block_size = 65,536, se formará un bloque comprimido por cada dos marcas. - -Estamos escribiendo una columna URL con el tipo String (tamaño promedio de 60 bytes por valor). Al escribir 8192 filas, el promedio será ligeramente inferior a 500 KB de datos. Como esto es más de 65,536, se formará un bloque comprimido para cada marca. En este caso, al leer datos del disco en el rango de una sola marca, los datos adicionales no se descomprimirán. - -Por lo general, no hay ninguna razón para cambiar esta configuración. - -## max_query_size {#settings-max_query_size} - -La parte máxima de una consulta que se puede llevar a la RAM para analizar con el analizador SQL. -La consulta INSERT también contiene datos para INSERT que es procesado por un analizador de secuencias independiente (que consume O(1) RAM), que no está incluido en esta restricción. - -Valor predeterminado: 256 KiB. - -## interactive_delay {#interactive-delay} - -El intervalo en microsegundos para comprobar si la ejecución de la solicitud se ha cancelado y enviar el progreso. - -Valor predeterminado: 100.000 (comprueba la cancelación y envía el progreso diez veces por segundo). - -## ¿Cómo puedo hacerlo? {#connect-timeout-receive-timeout-send-timeout} - -Tiempos de espera en segundos en el socket utilizado para comunicarse con el cliente. - -Valor predeterminado: 10, 300, 300. - -## Cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} - -Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. - -Valor predeterminado: 0 - -## poll_interval {#poll-interval} - -Bloquear en un bucle de espera durante el número especificado de segundos. - -Valor predeterminado: 10. - -## max_distributed_connections {#max-distributed-connections} - -El número máximo de conexiones simultáneas con servidores remotos para el procesamiento distribuido de una única consulta a una única tabla distribuida. Se recomienda establecer un valor no menor que el número de servidores en el clúster. - -Valor predeterminado: 1024. - -Los siguientes parámetros solo se usan al crear tablas distribuidas (y al iniciar un servidor), por lo que no hay ninguna razón para cambiarlas en tiempo de ejecución. - -## Distributed_connections_pool_size {#distributed-connections-pool-size} - -El número máximo de conexiones simultáneas con servidores remotos para el procesamiento distribuido de todas las consultas a una única tabla distribuida. Se recomienda establecer un valor no menor que el número de servidores en el clúster. - -Valor predeterminado: 1024. - -## Conecte_timeout_with_failover_ms {#connect-timeout-with-failover-ms} - -El tiempo de espera en milisegundos para conectarse a un servidor remoto para un motor de tablas distribuidas ‘shard’ y ‘replica’ secciones se utilizan en la definición de clúster. -Si no tiene éxito, se realizan varios intentos para conectarse a varias réplicas. - -Valor predeterminado: 50. - -## connections_with_failover_max_tries {#connections-with-failover-max-tries} - -El número máximo de intentos de conexión con cada réplica para el motor de tablas distribuidas. - -Valor predeterminado: 3. - -## extremo {#extremes} - -Ya sea para contar valores extremos (los mínimos y máximos en columnas de un resultado de consulta). Acepta 0 o 1. De forma predeterminada, 0 (deshabilitado). -Para obtener más información, consulte la sección “Extreme values”. - -## Use_uncompressed_cache {#setting-use_uncompressed_cache} - -Si se debe usar una memoria caché de bloques sin comprimir. Acepta 0 o 1. De forma predeterminada, 0 (deshabilitado). -El uso de la memoria caché sin comprimir (solo para tablas de la familia MergeTree) puede reducir significativamente la latencia y aumentar el rendimiento cuando se trabaja con un gran número de consultas cortas. Habilite esta configuración para los usuarios que envían solicitudes cortas frecuentes. También preste atención al [Uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. - -Para consultas que leen al menos un volumen algo grande de datos (un millón de filas o más), la memoria caché sin comprimir se desactiva automáticamente para ahorrar espacio para consultas realmente pequeñas. Esto significa que puede mantener el ‘use_uncompressed_cache’ ajuste siempre establecido en 1. - -## Reemplazar_running_query {#replace-running-query} - -Cuando se utiliza la interfaz HTTP, el ‘query_id’ parámetro puede ser pasado. Se trata de cualquier cadena que sirva como identificador de consulta. -Si una consulta del mismo usuario ‘query_id’ que ya existe en este momento, el comportamiento depende de la ‘replace_running_query’ parámetro. - -`0` (default) – Throw an exception (don't allow the query to run if a query with the same ‘query_id’ ya se está ejecutando). - -`1` – Cancel the old query and start running the new one. - -El Yandex.Metrica utiliza este parámetro establecido en 1 para implementar sugerencias para las condiciones de segmentación. Después de ingresar el siguiente carácter, si la consulta anterior aún no ha finalizado, debe cancelarse. - -## Nombre de la red inalámbrica (SSID): {#stream-flush-interval-ms} - -Funciona para tablas con streaming en el caso de un tiempo de espera, o cuando un subproceso genera [Max_insert_block_size](#settings-max_insert_block_size) filas. - -El valor predeterminado es 7500. - -Cuanto menor sea el valor, más a menudo los datos se vacían en la tabla. Establecer el valor demasiado bajo conduce a un rendimiento deficiente. - -## load_balancing {#settings-load_balancing} - -Especifica el algoritmo de selección de réplicas que se utiliza para el procesamiento de consultas distribuidas. - -ClickHouse admite los siguientes algoritmos para elegir réplicas: - -- [Aleatorio](#load_balancing-random) (predeterminada) -- [Nombre de host más cercano](#load_balancing-nearest_hostname) -- [En orden](#load_balancing-in_order) -- [Primero o aleatorio](#load_balancing-first_or_random) - -### Aleatorio (por defecto) {#load_balancing-random} - -``` sql -load_balancing = random -``` - -El número de errores se cuenta para cada réplica. La consulta se envía a la réplica con el menor número de errores, y si hay varios de estos, a cualquiera de ellos. -Desventajas: La proximidad del servidor no se tiene en cuenta; si las réplicas tienen datos diferentes, también obtendrá datos diferentes. - -### Nombre de host más cercano {#load_balancing-nearest_hostname} - -``` sql -load_balancing = nearest_hostname -``` - -The number of errors is counted for each replica. Every 5 minutes, the number of errors is integrally divided by 2. Thus, the number of errors is calculated for a recent time with exponential smoothing. If there is one replica with a minimal number of errors (i.e. errors occurred recently on the other replicas), the query is sent to it. If there are multiple replicas with the same minimal number of errors, the query is sent to the replica with a hostname that is most similar to the server's hostname in the config file (for the number of different characters in identical positions, up to the minimum length of both hostnames). - -Por ejemplo, example01-01-1 y example01-01-2.yandex.ru son diferentes en una posición, mientras que example01-01-1 y example01-02-2 difieren en dos lugares. -Este método puede parecer primitivo, pero no requiere datos externos sobre la topología de red, y no compara las direcciones IP, lo que sería complicado para nuestras direcciones IPv6. - -Por lo tanto, si hay réplicas equivalentes, se prefiere la más cercana por nombre. -También podemos suponer que al enviar una consulta al mismo servidor, en ausencia de fallas, una consulta distribuida también irá a los mismos servidores. Por lo tanto, incluso si se colocan datos diferentes en las réplicas, la consulta devolverá principalmente los mismos resultados. - -### En orden {#load_balancing-in_order} - -``` sql -load_balancing = in_order -``` - -Se accede a las réplicas con el mismo número de errores en el mismo orden en que se especifican en la configuración. -Este método es apropiado cuando se sabe exactamente qué réplica es preferible. - -### Primero o aleatorio {#load_balancing-first_or_random} - -``` sql -load_balancing = first_or_random -``` - -Este algoritmo elige la primera réplica del conjunto o una réplica aleatoria si la primera no está disponible. Es efectivo en configuraciones de topología de replicación cruzada, pero inútil en otras configuraciones. - -El `first_or_random` resuelve el problema del algoritmo `in_order` algoritmo. Con `in_order`, si una réplica se cae, la siguiente obtiene una carga doble mientras que las réplicas restantes manejan la cantidad habitual de tráfico. Cuando se utiliza el `first_or_random` algoritmo, la carga se distribuye uniformemente entre las réplicas que todavía están disponibles. - -## prefer_localhost_replica {#settings-prefer-localhost-replica} - -Habilita/deshabilita el uso preferible de la réplica localhost al procesar consultas distribuidas. - -Valores posibles: - -- 1 — ClickHouse always sends a query to the localhost replica if it exists. -- 0 — ClickHouse uses the balancing strategy specified by the [load_balancing](#settings-load_balancing) configuración. - -Valor predeterminado: 1. - -!!! warning "Advertencia" - Deshabilite esta configuración si usa [max_parallel_replicas](#settings-max_parallel_replicas). - -## totals_mode {#totals-mode} - -Cómo calcular TOTALS cuando HAVING está presente, así como cuando max_rows_to_group_by y group_by_overflow_mode = ‘any’ están presentes. -Vea la sección “WITH TOTALS modifier”. - -## totals_auto_threshold {#totals-auto-threshold} - -El umbral para `totals_mode = 'auto'`. -Vea la sección “WITH TOTALS modifier”. - -## max_parallel_replicas {#settings-max_parallel_replicas} - -El número máximo de réplicas para cada fragmento al ejecutar una consulta. -Para obtener coherencia (para obtener diferentes partes de la misma división de datos), esta opción solo funciona cuando se establece la clave de muestreo. -El retraso de réplica no está controlado. - -## compilar {#compile} - -Habilitar la compilación de consultas. De forma predeterminada, 0 (deshabilitado). - -La compilación solo se usa para parte de la canalización de procesamiento de consultas: para la primera etapa de agregación (GROUP BY). -Si se compiló esta parte de la canalización, la consulta puede ejecutarse más rápido debido a la implementación de ciclos cortos y a las llamadas de función agregadas en línea. La mejora del rendimiento máximo (hasta cuatro veces más rápido en casos excepcionales) se ve para consultas con múltiples funciones agregadas simples. Por lo general, la ganancia de rendimiento es insignificante. En casos muy raros, puede ralentizar la ejecución de la consulta. - -## min_count_to_compile {#min-count-to-compile} - -¿Cuántas veces usar potencialmente un fragmento de código compilado antes de ejecutar la compilación? Por defecto, 3. -For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. -Si el valor es 1 o más, la compilación se produce de forma asíncrona en un subproceso independiente. El resultado se utilizará tan pronto como esté listo, incluidas las consultas que se están ejecutando actualmente. - -Se requiere código compilado para cada combinación diferente de funciones agregadas utilizadas en la consulta y el tipo de claves en la cláusula GROUP BY. -The results of the compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. - -## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} - -Si el valor es true, los enteros aparecen entre comillas cuando se usan los formatos JSON\* Int64 y UInt64 (por compatibilidad con la mayoría de las implementaciones de JavaScript); de lo contrario, los enteros se generan sin las comillas. - -## Formato_csv_delimiter {#settings-format_csv_delimiter} - -El carácter interpretado como un delimitador en los datos CSV. De forma predeterminada, el delimitador es `,`. - -## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} - -Para el formato de entrada CSV, habilita o deshabilita el análisis de `NULL` como literal (sinónimo de `\N`). - -## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} - -Utilice el separador de línea de estilo DOS / Windows (CRLF) en CSV en lugar de estilo Unix (LF). - -## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} - -Utilice el separador de línea de estilo DOC / Windows (CRLF) en TSV en lugar del estilo Unix (LF). - -## insert_quorum {#settings-insert_quorum} - -Habilita las escrituras de quórum. - -- Si `insert_quorum < 2`, las escrituras de quórum están deshabilitadas. -- Si `insert_quorum >= 2`, las escrituras de quórum están habilitadas. - -Valor predeterminado: 0. - -Quorum escribe - -`INSERT` solo tiene éxito cuando ClickHouse logra escribir correctamente datos en el `insert_quorum` de réplicas durante el `insert_quorum_timeout`. Si por alguna razón el número de réplicas con escrituras exitosas no alcanza el `insert_quorum`, la escritura se considera fallida y ClickHouse eliminará el bloque insertado de todas las réplicas donde los datos ya se han escrito. - -Todas las réplicas del quórum son consistentes, es decir, contienen datos de todas las réplicas anteriores `INSERT` consulta. El `INSERT` la secuencia está linealizada. - -Al leer los datos escritos desde el `insert_quorum` usted puede utilizar el [select_sequential_consistency](#settings-select_sequential_consistency) opcion. - -ClickHouse genera una excepción - -- Si el número de réplicas disponibles en el momento de la consulta es `insert_quorum`. -- En un intento de escribir datos cuando el bloque anterior aún no se ha insertado en el `insert_quorum` de réplicas. Esta situación puede ocurrir si el usuario intenta realizar una `INSERT` antes de la anterior con el `insert_quorum` se ha completado. - -Ver también: - -- [insert_quorum_timeout](#settings-insert_quorum_timeout) -- [select_sequential_consistency](#settings-select_sequential_consistency) - -## insert_quorum_timeout {#settings-insert_quorum_timeout} - -Escribir en tiempo de espera de quórum en segundos. Si el tiempo de espera ha pasado y aún no se ha realizado ninguna escritura, ClickHouse generará una excepción y el cliente debe repetir la consulta para escribir el mismo bloque en la misma réplica o en cualquier otra réplica. - -Valor predeterminado: 60 segundos. - -Ver también: - -- [insert_quorum](#settings-insert_quorum) -- [select_sequential_consistency](#settings-select_sequential_consistency) - -## select_sequential_consistency {#settings-select_sequential_consistency} - -Habilita o deshabilita la coherencia secuencial para `SELECT` consulta: - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 0. - -Uso - -Cuando se habilita la coherencia secuencial, ClickHouse permite al cliente ejecutar el `SELECT` consulta sólo para aquellas réplicas que contienen datos de todas las `INSERT` consultas ejecutadas con `insert_quorum`. Si el cliente hace referencia a una réplica parcial, ClickHouse generará una excepción. La consulta SELECT no incluirá datos que aún no se hayan escrito en el quórum de réplicas. - -Ver también: - -- [insert_quorum](#settings-insert_quorum) -- [insert_quorum_timeout](#settings-insert_quorum_timeout) - -## insert_deduplicate {#settings-insert-deduplicate} - -Habilita o deshabilita la desduplicación de bloques `INSERT` (para tablas replicadas\* - -Valores posibles: - -- 0 — Disabled. -- 1 — Enabled. - -Valor predeterminado: 1. - -De forma predeterminada, los bloques insertados en tablas replicadas `INSERT` declaración se deduplican (ver [Replicación de datos](../../engines/table-engines/mergetree-family/replication.md)). - -## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} - -Habilita o deshabilita la comprobación de desduplicación para las vistas materializadas que reciben datos de tablas replicadas\*. - -Valores posibles: - - 0 — Disabled. - 1 — Enabled. - -Valor predeterminado: 0. - -Uso - -De forma predeterminada, la desduplicación no se realiza para las vistas materializadas, sino que se realiza en sentido ascendente, en la tabla de origen. -Si se omite un bloque INSERTed debido a la desduplicación en la tabla de origen, no habrá inserción en las vistas materializadas adjuntas. Este comportamiento existe para permitir la inserción de datos altamente agregados en vistas materializadas, para los casos en que los bloques insertados son los mismos después de la agregación de vistas materializadas pero derivados de diferentes INSERT en la tabla de origen. -Al mismo tiempo, este comportamiento “breaks” `INSERT` idempotencia. Si una `INSERT` en la mesa principal fue exitoso y `INSERT` into a materialized view failed (e.g. because of communication failure with Zookeeper) a client will get an error and can retry the operation. However, the materialized view won't receive the second insert because it will be discarded by deduplication in the main (source) table. The setting `deduplicate_blocks_in_dependent_materialized_views` permite cambiar este comportamiento. Al reintentar, una vista materializada recibirá la inserción de repetición y realizará la comprobación de desduplicación por sí misma, -ignorando el resultado de la comprobación para la tabla de origen, e insertará filas perdidas debido a la primera falla. - -## Método de codificación de datos: {#settings-max-network-bytes} - -Limita el volumen de datos (en bytes) que se recibe o se transmite a través de la red al ejecutar una consulta. Esta configuración se aplica a cada consulta individual. - -Valores posibles: - -- Entero positivo. -- 0 — Data volume control is disabled. - -Valor predeterminado: 0. - -## Método de codificación de datos: {#settings-max-network-bandwidth} - -Limita la velocidad del intercambio de datos a través de la red en bytes por segundo. Esta configuración se aplica a todas las consultas. - -Valores posibles: - -- Entero positivo. -- 0 — Bandwidth control is disabled. - -Valor predeterminado: 0. - -## Todos los derechos reservados {#settings-max-network-bandwidth-for-user} - -Limita la velocidad del intercambio de datos a través de la red en bytes por segundo. Esta configuración se aplica a todas las consultas que se ejecutan simultáneamente realizadas por un único usuario. - -Valores posibles: - -- Entero positivo. -- 0 — Control of the data speed is disabled. - -Valor predeterminado: 0. - -## Todos los derechos reservados {#settings-max-network-bandwidth-for-all-users} - -Limita la velocidad a la que se intercambian datos a través de la red en bytes por segundo. Esta configuración se aplica a todas las consultas que se ejecutan simultáneamente en el servidor. - -Valores posibles: - -- Entero positivo. -- 0 — Control of the data speed is disabled. - -Valor predeterminado: 0. - -## count_distinct_implementation {#settings-count_distinct_implementation} - -Especifica cuál de las `uniq*` se deben utilizar para realizar el [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference.md#agg_function-count) construcción. - -Valores posibles: - -- [uniq](../../sql-reference/aggregate-functions/reference.md#agg_function-uniq) -- [uniqCombined](../../sql-reference/aggregate-functions/reference.md#agg_function-uniqcombined) -- [UniqCombined64](../../sql-reference/aggregate-functions/reference.md#agg_function-uniqcombined64) -- [uniqHLL12](../../sql-reference/aggregate-functions/reference.md#agg_function-uniqhll12) -- [uniqExact](../../sql-reference/aggregate-functions/reference.md#agg_function-uniqexact) - -Valor predeterminado: `uniqExact`. - -## skip_unavailable_shards {#settings-skip_unavailable_shards} - -Habilita o deshabilita la omisión silenciosa de fragmentos no disponibles. - -El fragmento se considera no disponible si todas sus réplicas no están disponibles. Una réplica no está disponible en los siguientes casos: - -- ClickHouse no puede conectarse a la réplica por ningún motivo. - - Al conectarse a una réplica, ClickHouse realiza varios intentos. Si todos estos intentos fallan, la réplica se considera que no está disponible. - -- La réplica no se puede resolver a través de DNS. - - Si el nombre de host de la réplica no se puede resolver a través de DNS, puede indicar las siguientes situaciones: - - - El host de Replica no tiene registro DNS. Puede ocurrir en sistemas con DNS dinámico, por ejemplo, [Kubernetes](https://kubernetes.io), donde los nodos pueden ser irresolubles durante el tiempo de inactividad, y esto no es un error. - - - Error de configuración. El archivo de configuración de ClickHouse contiene un nombre de host incorrecto. - -Valores posibles: - -- 1 — skipping enabled. - - Si un fragmento no está disponible, ClickHouse devuelve un resultado basado en datos parciales y no informa de problemas de disponibilidad de nodos. - -- 0 — skipping disabled. - - Si un fragmento no está disponible, ClickHouse produce una excepción. - -Valor predeterminado: 0. - -## Optize_skip_unused_shards {#settings-optimize_skip_unused_shards} - -Habilita o deshabilita la omisión de fragmentos no utilizados para las consultas SELECT que tienen la condición de clave de fragmentación en PREWHERE / WHERE (supone que los datos se distribuyen mediante clave de fragmentación, de lo contrario no hacer nada). - -Valor predeterminado: 0 - -## Fuerza_optimize_skip_unused_shards {#settings-force_optimize_skip_unused_shards} - -Habilita o deshabilita la ejecución de consultas si [`optimize_skip_unused_shards`](#settings-optimize_skip_unused_shards) no es posible omitir fragmentos no utilizados. Si la omisión no es posible y la configuración está habilitada, se lanzará una excepción. - -Valores posibles: - -- 0 - Discapacitados (no lanza) -- 1: deshabilite la ejecución de consultas solo si la tabla tiene una clave de fragmentación -- 2: deshabilita la ejecución de consultas independientemente de que se haya definido la clave de fragmentación para la tabla - -Valor predeterminado: 0 - -## Optize_throw_if_noop {#setting-optimize_throw_if_noop} - -Habilita o deshabilita el lanzamiento de una excepción [OPTIMIZE](../../sql-reference/statements/misc.md#misc_operations-optimize) la consulta no realizó una fusión. - -Predeterminada, `OPTIMIZE` devuelve con éxito incluso si no hizo nada. Esta configuración le permite diferenciar estas situaciones y obtener el motivo en un mensaje de excepción. - -Valores posibles: - -- 1 — Throwing an exception is enabled. -- 0 — Throwing an exception is disabled. - -Valor predeterminado: 0. - -## distributed_replica_error_half_life {#settings-distributed_replica_error_half_life} - -- Tipo: segundos -- Valor predeterminado: 60 segundos - -Controla la rapidez con la que se ponen a cero los errores en las tablas distribuidas. Si una réplica no está disponible durante algún tiempo, acumula 5 errores y distribut_replica_error_half_life se establece en 1 segundo, la réplica se considera normal 3 segundos después del último error. - -Ver también: - -- [Motor de tabla distribuido](../../engines/table-engines/special/distributed.md) -- [distributed_replica_error_cap](#settings-distributed_replica_error_cap) - -## distributed_replica_error_cap {#settings-distributed_replica_error_cap} - -- Tipo: unsigned int -- Valor predeterminado: 1000 - -El recuento de errores de cada réplica está limitado a este valor, lo que impide que una sola réplica acumule demasiados errores. - -Ver también: - -- [Motor de tabla distribuido](../../engines/table-engines/special/distributed.md) -- [distributed_replica_error_half_life](#settings-distributed_replica_error_half_life) - -## Distributed_directory_monitor_sleep_time_ms {#distributed_directory_monitor_sleep_time_ms} - -Intervalo base para el [Distribuido](../../engines/table-engines/special/distributed.md) motor de tabla para enviar datos. El intervalo real crece exponencialmente en caso de errores. - -Valores posibles: - -- Un número entero positivo de milisegundos. - -Valor predeterminado: 100 milisegundos. - -## Distributed_directory_monitor_max_sleep_time_ms {#distributed_directory_monitor_max_sleep_time_ms} - -Intervalo máximo para el [Distribuido](../../engines/table-engines/special/distributed.md) motor de tabla para enviar datos. Limita el crecimiento exponencial del intervalo establecido en el [Distributed_directory_monitor_sleep_time_ms](#distributed_directory_monitor_sleep_time_ms) configuración. - -Valores posibles: - -- Un número entero positivo de milisegundos. - -Valor predeterminado: 30000 milisegundos (30 segundos). - -## distributed_directory_monitor_batch_inserts {#distributed_directory_monitor_batch_inserts} - -Habilita/deshabilita el envío de datos insertados en lotes. - -Cuando el envío por lotes está habilitado, el [Distribuido](../../engines/table-engines/special/distributed.md) El motor de tabla intenta enviar varios archivos de datos insertados en una operación en lugar de enviarlos por separado. El envío por lotes mejora el rendimiento del clúster al utilizar mejor los recursos del servidor y de la red. - -Valores posibles: - -- 1 — Enabled. -- 0 — Disabled. - -Valor predeterminado: 0. - -## os_thread_priority {#setting-os-thread-priority} - -Establece la prioridad ([agradable](https://en.wikipedia.org/wiki/Nice_(Unix))) para subprocesos que ejecutan consultas. El programador del sistema operativo considera esta prioridad al elegir el siguiente hilo para ejecutar en cada núcleo de CPU disponible. - -!!! warning "Advertencia" - Para utilizar esta configuración, debe establecer el `CAP_SYS_NICE` capacidad. El `clickhouse-server` paquete lo configura durante la instalación. Algunos entornos virtuales no le permiten establecer `CAP_SYS_NICE` capacidad. En este caso, `clickhouse-server` muestra un mensaje al respecto al principio. - -Valores posibles: - -- Puede establecer valores en el rango `[-20, 19]`. - -Los valores más bajos significan mayor prioridad. Hilos con bajo `nice` Los valores de prioridad se ejecutan con más frecuencia que los subprocesos con valores altos. Los valores altos son preferibles para consultas no interactivas de larga ejecución porque les permite renunciar rápidamente a recursos en favor de consultas interactivas cortas cuando llegan. - -Valor predeterminado: 0. - -## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} - -Establece el período para un temporizador de reloj real del [perfilador de consultas](../../operations/optimizing-performance/sampling-query-profiler.md). El temporizador de reloj real cuenta el tiempo del reloj de pared. - -Valores posibles: - -- Número entero positivo, en nanosegundos. - - Valores recomendados: - - - 10000000 (100 times a second) nanoseconds and less for single queries. - - 1000000000 (once a second) for cluster-wide profiling. - -- 0 para apagar el temporizador. - -Tipo: [UInt64](../../sql-reference/data-types/int-uint.md). - -Valor predeterminado: 1000000000 nanosegundos (una vez por segundo). - -Ver también: - -- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) - -## Los resultados de la prueba {#query_profiler_cpu_time_period_ns} - -Establece el período para un temporizador de reloj de CPU [perfilador de consultas](../../operations/optimizing-performance/sampling-query-profiler.md). Este temporizador solo cuenta el tiempo de CPU. - -Valores posibles: - -- Un número entero positivo de nanosegundos. - - Valores recomendados: - - - 10000000 (100 times a second) nanoseconds and more for single queries. - - 1000000000 (once a second) for cluster-wide profiling. - -- 0 para apagar el temporizador. - -Tipo: [UInt64](../../sql-reference/data-types/int-uint.md). - -Valor predeterminado: 1000000000 nanosegundos. - -Ver también: - -- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) - -## allow_introspection_functions {#settings-allow_introspection_functions} - -Habilita deshabilita [funciones de introspecciones](../../sql-reference/functions/introspection.md) para la creación de perfiles de consultas. - -Valores posibles: - -- 1 — Introspection functions enabled. -- 0 — Introspection functions disabled. - -Valor predeterminado: 0. - -**Ver también** - -- [Analizador de consultas de muestreo](../optimizing-performance/sampling-query-profiler.md) -- Tabla del sistema [trace_log](../../operations/system-tables.md#system_tables-trace_log) - -## input_format_parallel_parsing {#input-format-parallel-parsing} - -- Tipo: bool -- Valor predeterminado: True - -Habilitar el análisis paralelo de los formatos de datos para preservar el orden. Solo se admite para los formatos TSV, TKSV, CSV y JSONEachRow. - -## También puede utilizar los siguientes métodos de envío: {#min-chunk-bytes-for-parallel-parsing} - -- Tipo: unsigned int -- Valor predeterminado: 1 MiB - -El tamaño mínimo de fragmento en bytes, que cada subproceso analizará en paralelo. - -## Sistema abierto {#settings-output_format_avro_codec} - -Establece el códec de compresión utilizado para el archivo Avro de salida. - -Tipo: cadena - -Valores posibles: - -- `null` — No compression -- `deflate` — Compress with Deflate (zlib) -- `snappy` — Compress with [Rápido](https://google.github.io/snappy/) - -Valor predeterminado: `snappy` (si está disponible) o `deflate`. - -## Sistema abierto {#settings-output_format_avro_sync_interval} - -Establece el tamaño mínimo de datos (en bytes) entre los marcadores de sincronización para el archivo Avro de salida. - -Tipo: unsigned int - -Valores posibles: 32 (32 bytes) - 1073741824 (1 GiB) - -Valor predeterminado: 32768 (32 KiB) - -## Todos los derechos reservados {#settings-format_avro_schema_registry_url} - -Establece la URL del Registro de esquemas confluentes para usar con [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) formato - -Tipo: URL - -Valor predeterminado: Vacío - -## background_pool_size {#background_pool_size} - -Establece el número de subprocesos que realizan operaciones en segundo plano en motores de tabla (por ejemplo, fusiona [Motor MergeTree](../../engines/table-engines/mergetree-family/index.md) tabla). Esta configuración se aplica al inicio del servidor ClickHouse y no se puede cambiar en una sesión de usuario. Al ajustar esta configuración, puede administrar la carga de la CPU y el disco. Un tamaño de grupo más pequeño utiliza menos recursos de CPU y disco, pero los procesos en segundo plano avanzan más lentamente, lo que eventualmente podría afectar el rendimiento de la consulta. - -Valores posibles: - -- Cualquier entero positivo. - -Valor predeterminado: 16. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/es/operations/system-tables.md b/docs/es/operations/system-tables.md deleted file mode 100644 index 18e7f7227da..00000000000 --- a/docs/es/operations/system-tables.md +++ /dev/null @@ -1,1168 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 52 -toc_title: Tablas del sistema ---- - -# Tablas del sistema {#system-tables} - -Las tablas del sistema se utilizan para implementar parte de la funcionalidad del sistema y para proporcionar acceso a información sobre cómo funciona el sistema. -No puede eliminar una tabla del sistema (pero puede realizar DETACH). -Las tablas del sistema no tienen archivos con datos en el disco o archivos con metadatos. El servidor crea todas las tablas del sistema cuando se inicia. -Las tablas del sistema son de solo lectura. -Están ubicados en el ‘system’ base. - -## sistema.asynchronous_metrics {#system_tables-asynchronous_metrics} - -Contiene métricas que se calculan periódicamente en segundo plano. Por ejemplo, la cantidad de RAM en uso. - -Columna: - -- `metric` ([Cadena](../sql-reference/data-types/string.md)) — Metric name. -- `value` ([Float64](../sql-reference/data-types/float.md)) — Metric value. - -**Ejemplo** - -``` sql -SELECT * FROM system.asynchronous_metrics LIMIT 10 -``` - -``` text -┌─metric──────────────────────────────────┬──────value─┐ -│ jemalloc.background_thread.run_interval │ 0 │ -│ jemalloc.background_thread.num_runs │ 0 │ -│ jemalloc.background_thread.num_threads │ 0 │ -│ jemalloc.retained │ 422551552 │ -│ jemalloc.mapped │ 1682989056 │ -│ jemalloc.resident │ 1656446976 │ -│ jemalloc.metadata_thp │ 0 │ -│ jemalloc.metadata │ 10226856 │ -│ UncompressedCacheCells │ 0 │ -│ MarkCacheFiles │ 0 │ -└─────────────────────────────────────────┴────────────┘ -``` - -**Ver también** - -- [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. -- [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. -- [sistema.evento](#system_tables-events) — Contains a number of events that have occurred. -- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - -## sistema.Cluster {#system-clusters} - -Contiene información sobre los clústeres disponibles en el archivo de configuración y los servidores que contienen. - -Columna: - -- `cluster` (String) — The cluster name. -- `shard_num` (UInt32) — The shard number in the cluster, starting from 1. -- `shard_weight` (UInt32) — The relative weight of the shard when writing data. -- `replica_num` (UInt32) — The replica number in the shard, starting from 1. -- `host_name` (String) — The host name, as specified in the config. -- `host_address` (String) — The host IP address obtained from DNS. -- `port` (UInt16) — The port to use for connecting to the server. -- `user` (String) — The name of the user for connecting to the server. -- `errors_count` (UInt32): número de veces que este host no pudo alcanzar la réplica. -- `estimated_recovery_time` (UInt32): quedan segundos hasta que el recuento de errores de réplica se ponga a cero y se considere que vuelve a la normalidad. - -Tenga en cuenta que `errors_count` se actualiza una vez por consulta al clúster, pero `estimated_recovery_time` se vuelve a calcular bajo demanda. Entonces podría haber un caso distinto de cero `errors_count` y cero `estimated_recovery_time`, esa próxima consulta será cero `errors_count` e intente usar la réplica como si no tuviera errores. - -**Ver también** - -- [Motor de tabla distribuido](../engines/table-engines/special/distributed.md) -- [distributed_replica_error_cap configuración](settings/settings.md#settings-distributed_replica_error_cap) -- [distributed_replica_error_half_life configuración](settings/settings.md#settings-distributed_replica_error_half_life) - -## sistema.columna {#system-columns} - -Contiene información sobre las columnas de todas las tablas. - -Puede utilizar esta tabla para obtener información similar a la [DESCRIBE TABLE](../sql-reference/statements/misc.md#misc-describe-table) consulta, pero para varias tablas a la vez. - -El `system.columns` tabla contiene las siguientes columnas (el tipo de columna se muestra entre corchetes): - -- `database` (String) — Database name. -- `table` (String) — Table name. -- `name` (String) — Column name. -- `type` (String) — Column type. -- `default_kind` (String) — Expression type (`DEFAULT`, `MATERIALIZED`, `ALIAS`) para el valor predeterminado, o una cadena vacía si no está definida. -- `default_expression` (String) — Expression for the default value, or an empty string if it is not defined. -- `data_compressed_bytes` (UInt64) — The size of compressed data, in bytes. -- `data_uncompressed_bytes` (UInt64) — The size of decompressed data, in bytes. -- `marks_bytes` (UInt64) — The size of marks, in bytes. -- `comment` (String) — Comment on the column, or an empty string if it is not defined. -- `is_in_partition_key` (UInt8) — Flag that indicates whether the column is in the partition expression. -- `is_in_sorting_key` (UInt8) — Flag that indicates whether the column is in the sorting key expression. -- `is_in_primary_key` (UInt8) — Flag that indicates whether the column is in the primary key expression. -- `is_in_sampling_key` (UInt8) — Flag that indicates whether the column is in the sampling key expression. - -## sistema.colaborador {#system-contributors} - -Contiene información sobre los colaboradores. Todos los constributores en orden aleatorio. El orden es aleatorio en el momento de la ejecución de la consulta. - -Columna: - -- `name` (String) — Contributor (author) name from git log. - -**Ejemplo** - -``` sql -SELECT * FROM system.contributors LIMIT 10 -``` - -``` text -┌─name─────────────┐ -│ Olga Khvostikova │ -│ Max Vetrov │ -│ LiuYangkuan │ -│ svladykin │ -│ zamulla │ -│ Šimon Podlipský │ -│ BayoNet │ -│ Ilya Khomutov │ -│ Amy Krishnevsky │ -│ Loud_Scream │ -└──────────────────┘ -``` - -Para descubrirlo en la tabla, use una consulta: - -``` sql -SELECT * FROM system.contributors WHERE name='Olga Khvostikova' -``` - -``` text -┌─name─────────────┐ -│ Olga Khvostikova │ -└──────────────────┘ -``` - -## sistema.base {#system-databases} - -Esta tabla contiene una sola columna String llamada ‘name’ – the name of a database. -Cada base de datos que el servidor conoce tiene una entrada correspondiente en la tabla. -Esta tabla del sistema se utiliza para implementar el `SHOW DATABASES` consulta. - -## sistema.detached_parts {#system_tables-detached_parts} - -Contiene información sobre piezas separadas de [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md) tabla. El `reason` columna especifica por qué se separó la pieza. Para las piezas separadas por el usuario, el motivo está vacío. Tales partes se pueden unir con [ALTER TABLE ATTACH PARTITION\|PART](../sql-reference/statements/alter.md#alter_attach-partition) comando. Para obtener la descripción de otras columnas, consulte [sistema.parte](#system_tables-parts). Si el nombre de la pieza no es válido, los valores de algunas columnas pueden ser `NULL`. Tales partes se pueden eliminar con [ALTER TABLE DROP DETACHED PART](../sql-reference/statements/alter.md#alter_drop-detached). - -## sistema.diccionario {#system_tables-dictionaries} - -Contiene información sobre [diccionarios externos](../sql-reference/dictionaries/external-dictionaries/external-dicts.md). - -Columna: - -- `database` ([Cadena](../sql-reference/data-types/string.md)) — Name of the database containing the dictionary created by DDL query. Empty string for other dictionaries. -- `name` ([Cadena](../sql-reference/data-types/string.md)) — [Nombre del diccionario](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md). -- `status` ([Enum8](../sql-reference/data-types/enum.md)) — Dictionary status. Possible values: - - `NOT_LOADED` — Dictionary was not loaded because it was not used. - - `LOADED` — Dictionary loaded successfully. - - `FAILED` — Unable to load the dictionary as a result of an error. - - `LOADING` — Dictionary is loading now. - - `LOADED_AND_RELOADING` — Dictionary is loaded successfully, and is being reloaded right now (frequent reasons: [SYSTEM RELOAD DICTIONARY](../sql-reference/statements/system.md#query_language-system-reload-dictionary) consulta, tiempo de espera, configuración del diccionario ha cambiado). - - `FAILED_AND_RELOADING` — Could not load the dictionary as a result of an error and is loading now. -- `origin` ([Cadena](../sql-reference/data-types/string.md)) — Path to the configuration file that describes the dictionary. -- `type` ([Cadena](../sql-reference/data-types/string.md)) — Type of a dictionary allocation. [Almacenamiento de diccionarios en la memoria](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). -- `key` — [Tipo de llave](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key): Clave numérica ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) or Сomposite key ([Cadena](../sql-reference/data-types/string.md)) — form “(type 1, type 2, …, type n)”. -- `attribute.names` ([Matriz](../sql-reference/data-types/array.md)([Cadena](../sql-reference/data-types/string.md))) — Array of [nombres de atributos](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) proporcionada por el diccionario. -- `attribute.types` ([Matriz](../sql-reference/data-types/array.md)([Cadena](../sql-reference/data-types/string.md))) — Corresponding array of [tipos de atributos](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) que son proporcionados por el diccionario. -- `bytes_allocated` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Amount of RAM allocated for the dictionary. -- `query_count` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of queries since the dictionary was loaded or since the last successful reboot. -- `hit_rate` ([Float64](../sql-reference/data-types/float.md)) — For cache dictionaries, the percentage of uses for which the value was in the cache. -- `element_count` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of items stored in the dictionary. -- `load_factor` ([Float64](../sql-reference/data-types/float.md)) — Percentage filled in the dictionary (for a hashed dictionary, the percentage filled in the hash table). -- `source` ([Cadena](../sql-reference/data-types/string.md)) — Text describing the [fuente de datos](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) para el diccionario. -- `lifetime_min` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [vida](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) del diccionario en la memoria, después de lo cual ClickHouse intenta volver a cargar el diccionario (si `invalidate_query` está configurado, entonces solo si ha cambiado). Establecer en segundos. -- `lifetime_max` ([UInt64](../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [vida](../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) del diccionario en la memoria, después de lo cual ClickHouse intenta volver a cargar el diccionario (si `invalidate_query` está configurado, entonces solo si ha cambiado). Establecer en segundos. -- `loading_start_time` ([FechaHora](../sql-reference/data-types/datetime.md)) — Start time for loading the dictionary. -- `last_successful_update_time` ([FechaHora](../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with external sources and investigate causes. -- `loading_duration` ([Float32](../sql-reference/data-types/float.md)) — Duration of a dictionary loading. -- `last_exception` ([Cadena](../sql-reference/data-types/string.md)) — Text of the error that occurs when creating or reloading the dictionary if the dictionary couldn't be created. - -**Ejemplo** - -Configurar el diccionario. - -``` sql -CREATE DICTIONARY dictdb.dict -( - `key` Int64 DEFAULT -1, - `value_default` String DEFAULT 'world', - `value_expression` String DEFAULT 'xxx' EXPRESSION 'toString(127 * 172)' -) -PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dicttbl' DB 'dictdb')) -LIFETIME(MIN 0 MAX 1) -LAYOUT(FLAT()) -``` - -Asegúrese de que el diccionario esté cargado. - -``` sql -SELECT * FROM system.dictionaries -``` - -``` text -┌─database─┬─name─┬─status─┬─origin──────┬─type─┬─key────┬─attribute.names──────────────────────┬─attribute.types─────┬─bytes_allocated─┬─query_count─┬─hit_rate─┬─element_count─┬───────────load_factor─┬─source─────────────────────┬─lifetime_min─┬─lifetime_max─┬──loading_start_time─┌──last_successful_update_time─┬──────loading_duration─┬─last_exception─┐ -│ dictdb │ dict │ LOADED │ dictdb.dict │ Flat │ UInt64 │ ['value_default','value_expression'] │ ['String','String'] │ 74032 │ 0 │ 1 │ 1 │ 0.0004887585532746823 │ ClickHouse: dictdb.dicttbl │ 0 │ 1 │ 2020-03-04 04:17:34 │ 2020-03-04 04:30:34 │ 0.002 │ │ -└──────────┴──────┴────────┴─────────────┴──────┴────────┴──────────────────────────────────────┴─────────────────────┴─────────────────┴─────────────┴──────────┴───────────────┴───────────────────────┴────────────────────────────┴──────────────┴──────────────┴─────────────────────┴──────────────────────────────┘───────────────────────┴────────────────┘ -``` - -## sistema.evento {#system_tables-events} - -Contiene información sobre el número de eventos que se han producido en el sistema. Por ejemplo, en la tabla, puede encontrar cuántos `SELECT` las consultas se procesaron desde que se inició el servidor ClickHouse. - -Columna: - -- `event` ([Cadena](../sql-reference/data-types/string.md)) — Event name. -- `value` ([UInt64](../sql-reference/data-types/int-uint.md)) — Number of events occurred. -- `description` ([Cadena](../sql-reference/data-types/string.md)) — Event description. - -**Ejemplo** - -``` sql -SELECT * FROM system.events LIMIT 5 -``` - -``` text -┌─event─────────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │ -│ SelectQuery │ 8 │ Same as Query, but only for SELECT queries. │ -│ FileOpen │ 73 │ Number of files opened. │ -│ ReadBufferFromFileDescriptorRead │ 155 │ Number of reads (read/pread) from a file descriptor. Does not include sockets. │ -│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │ -└───────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -**Ver también** - -- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. -- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. -- [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. - -## sistema.función {#system-functions} - -Contiene información sobre funciones normales y agregadas. - -Columna: - -- `name`(`String`) – The name of the function. -- `is_aggregate`(`UInt8`) — Whether the function is aggregate. - -## sistema.graphite_retentions {#system-graphite-retentions} - -Contiene información sobre los parámetros [graphite_rollup](server-configuration-parameters/settings.md#server_configuration_parameters-graphite) que se utilizan en tablas con [\*GraphiteMergeTree](../engines/table-engines/mergetree-family/graphitemergetree.md) motor. - -Columna: - -- `config_name` (Cadena) - `graphite_rollup` nombre del parámetro. -- `regexp` (Cadena) - Un patrón para el nombre de la métrica. -- `function` (String) - El nombre de la función de agregación. -- `age` (UInt64) - La edad mínima de los datos en segundos. -- `precision` (UInt64) - Cómo definir con precisión la edad de los datos en segundos. -- `priority` (UInt16) - Prioridad de patrón. -- `is_default` (UInt8) - Si el patrón es el predeterminado. -- `Tables.database` (Array(String)) - Matriz de nombres de tablas de base de datos que utilizan `config_name` parámetro. -- `Tables.table` (Array(String)) - Matriz de nombres de tablas que utilizan `config_name` parámetro. - -## sistema.fusionar {#system-merges} - -Contiene información sobre fusiones y mutaciones de piezas actualmente en proceso para tablas de la familia MergeTree. - -Columna: - -- `database` (String) — The name of the database the table is in. -- `table` (String) — Table name. -- `elapsed` (Float64) — The time elapsed (in seconds) since the merge started. -- `progress` (Float64) — The percentage of completed work from 0 to 1. -- `num_parts` (UInt64) — The number of pieces to be merged. -- `result_part_name` (String) — The name of the part that will be formed as the result of merging. -- `is_mutation` (UInt8) - 1 si este proceso es una mutación parte. -- `total_size_bytes_compressed` (UInt64) — The total size of the compressed data in the merged chunks. -- `total_size_marks` (UInt64) — The total number of marks in the merged parts. -- `bytes_read_uncompressed` (UInt64) — Number of bytes read, uncompressed. -- `rows_read` (UInt64) — Number of rows read. -- `bytes_written_uncompressed` (UInt64) — Number of bytes written, uncompressed. -- `rows_written` (UInt64) — Number of rows written. - -## sistema.métricas {#system_tables-metrics} - -Contiene métricas que pueden calcularse instantáneamente o tener un valor actual. Por ejemplo, el número de consultas procesadas simultáneamente o el retraso de réplica actual. Esta tabla está siempre actualizada. - -Columna: - -- `metric` ([Cadena](../sql-reference/data-types/string.md)) — Metric name. -- `value` ([Int64](../sql-reference/data-types/int-uint.md)) — Metric value. -- `description` ([Cadena](../sql-reference/data-types/string.md)) — Metric description. - -La lista de métricas admitidas que puede encontrar en el [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) archivo fuente de ClickHouse. - -**Ejemplo** - -``` sql -SELECT * FROM system.metrics LIMIT 10 -``` - -``` text -┌─metric─────────────────────┬─value─┬─description──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Query │ 1 │ Number of executing queries │ -│ Merge │ 0 │ Number of executing background merges │ -│ PartMutation │ 0 │ Number of mutations (ALTER DELETE/UPDATE) │ -│ ReplicatedFetch │ 0 │ Number of data parts being fetched from replicas │ -│ ReplicatedSend │ 0 │ Number of data parts being sent to replicas │ -│ ReplicatedChecks │ 0 │ Number of data parts checking for consistency │ -│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │ -│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │ -│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │ -│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │ -└────────────────────────────┴───────┴──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -**Ver también** - -- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [sistema.evento](#system_tables-events) — Contains a number of events that occurred. -- [sistema.metric_log](#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. -- [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. - -## sistema.metric_log {#system_tables-metric_log} - -Contiene el historial de valores de métricas de tablas `system.metrics` y `system.events`, periódicamente enjuagado al disco. -Para activar la recopilación de historial de métricas en `system.metric_log`, crear `/etc/clickhouse-server/config.d/metric_log.xml` con el siguiente contenido: - -``` xml - - - system - metric_log
- 7500 - 1000 -
-
-``` - -**Ejemplo** - -``` sql -SELECT * FROM system.metric_log LIMIT 1 FORMAT Vertical; -``` - -``` text -Row 1: -────── -event_date: 2020-02-18 -event_time: 2020-02-18 07:15:33 -milliseconds: 554 -ProfileEvent_Query: 0 -ProfileEvent_SelectQuery: 0 -ProfileEvent_InsertQuery: 0 -ProfileEvent_FileOpen: 0 -ProfileEvent_Seek: 0 -ProfileEvent_ReadBufferFromFileDescriptorRead: 1 -ProfileEvent_ReadBufferFromFileDescriptorReadFailed: 0 -ProfileEvent_ReadBufferFromFileDescriptorReadBytes: 0 -ProfileEvent_WriteBufferFromFileDescriptorWrite: 1 -ProfileEvent_WriteBufferFromFileDescriptorWriteFailed: 0 -ProfileEvent_WriteBufferFromFileDescriptorWriteBytes: 56 -... -CurrentMetric_Query: 0 -CurrentMetric_Merge: 0 -CurrentMetric_PartMutation: 0 -CurrentMetric_ReplicatedFetch: 0 -CurrentMetric_ReplicatedSend: 0 -CurrentMetric_ReplicatedChecks: 0 -... -``` - -**Ver también** - -- [sistema.asynchronous_metrics](#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [sistema.evento](#system_tables-events) — Contains a number of events that occurred. -- [sistema.métricas](#system_tables-metrics) — Contains instantly calculated metrics. -- [Monitoreo](monitoring.md) — Base concepts of ClickHouse monitoring. - -## sistema.numero {#system-numbers} - -Esta tabla contiene una única columna UInt64 llamada ‘number’ que contiene casi todos los números naturales a partir de cero. -Puede usar esta tabla para pruebas, o si necesita hacer una búsqueda de fuerza bruta. -Las lecturas de esta tabla no están paralelizadas. - -## sistema.Números_mt {#system-numbers-mt} - -Lo mismo que ‘system.numbers’ pero las lecturas están paralelizadas. Los números se pueden devolver en cualquier orden. -Se utiliza para pruebas. - -## sistema.una {#system-one} - -Esta tabla contiene una sola fila con una ‘dummy’ Columna UInt8 que contiene el valor 0. -Esta tabla se utiliza si una consulta SELECT no especifica la cláusula FROM. -Esto es similar a la tabla DUAL que se encuentra en otros DBMS. - -## sistema.parte {#system_tables-parts} - -Contiene información sobre partes de [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md) tabla. - -Cada fila describe una parte de datos. - -Columna: - -- `partition` (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../sql-reference/statements/alter.md#query_language_queries_alter) consulta. - - Formato: - - - `YYYYMM` para la partición automática por mes. - - `any_string` al particionar manualmente. - -- `name` (`String`) – Name of the data part. - -- `active` (`UInt8`) – Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's deleted. Inactive data parts remain after merging. - -- `marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` por la granularidad del índice (generalmente 8192) (esta sugerencia no funciona para la granularidad adaptativa). - -- `rows` (`UInt64`) – The number of rows. - -- `bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes. - -- `data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included. - -- `data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. - -- `marks_bytes` (`UInt64`) – The size of the file with marks. - -- `modification_time` (`DateTime`) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.\| - -- `remove_time` (`DateTime`) – The time when the data part became inactive. - -- `refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. - -- `min_date` (`Date`) – The minimum value of the date key in the data part. - -- `max_date` (`Date`) – The maximum value of the date key in the data part. - -- `min_time` (`DateTime`) – The minimum value of the date and time key in the data part. - -- `max_time`(`DateTime`) – The maximum value of the date and time key in the data part. - -- `partition_id` (`String`) – ID of the partition. - -- `min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging. - -- `max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging. - -- `level` (`UInt32`) – Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts. - -- `data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than `data_version`). - -- `primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values. - -- `primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values. - -- `is_frozen` (`UInt8`) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../sql-reference/statements/alter.md#alter_freeze-partition) - -- `database` (`String`) – Name of the database. - -- `table` (`String`) – Name of the table. - -- `engine` (`String`) – Name of the table engine without parameters. - -- `path` (`String`) – Absolute path to the folder with data part files. - -- `disk` (`String`) – Name of a disk that stores the data part. - -- `hash_of_all_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) de archivos comprimidos. - -- `hash_of_uncompressed_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) de archivos sin comprimir (archivos con marcas, archivo de índice, etc.). - -- `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../sql-reference/functions/hash-functions.md#hash_functions-siphash128) de datos en los archivos comprimidos como si estuvieran descomprimidos. - -- `bytes` (`UInt64`) – Alias for `bytes_on_disk`. - -- `marks_size` (`UInt64`) – Alias for `marks_bytes`. - -## sistema.part_log {#system_tables-part-log} - -El `system.part_log` se crea sólo si el [part_log](server-configuration-parameters/settings.md#server_configuration_parameters-part-log) se especifica la configuración del servidor. - -Esta tabla contiene información sobre eventos que ocurrieron con [partes de datos](../engines/table-engines/mergetree-family/custom-partitioning-key.md) en el [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md) tablas familiares, como agregar o fusionar datos. - -El `system.part_log` contiene las siguientes columnas: - -- `event_type` (Enum) — Type of the event that occurred with the data part. Can have one of the following values: - - `NEW_PART` — Inserting of a new data part. - - `MERGE_PARTS` — Merging of data parts. - - `DOWNLOAD_PART` — Downloading a data part. - - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../sql-reference/statements/alter.md#alter_detach-partition). - - `MUTATE_PART` — Mutating of a data part. - - `MOVE_PART` — Moving the data part from the one disk to another one. -- `event_date` (Date) — Event date. -- `event_time` (DateTime) — Event time. -- `duration_ms` (UInt64) — Duration. -- `database` (String) — Name of the database the data part is in. -- `table` (String) — Name of the table the data part is in. -- `part_name` (String) — Name of the data part. -- `partition_id` (String) — ID of the partition that the data part was inserted to. The column takes the ‘all’ valor si la partición es por `tuple()`. -- `rows` (UInt64) — The number of rows in the data part. -- `size_in_bytes` (UInt64) — Size of the data part in bytes. -- `merged_from` (Array(String)) — An array of names of the parts which the current part was made up from (after the merge). -- `bytes_uncompressed` (UInt64) — Size of uncompressed bytes. -- `read_rows` (UInt64) — The number of rows was read during the merge. -- `read_bytes` (UInt64) — The number of bytes was read during the merge. -- `error` (UInt16) — The code number of the occurred error. -- `exception` (String) — Text message of the occurred error. - -El `system.part_log` se crea después de la primera inserción de datos `MergeTree` tabla. - -## sistema.procesa {#system_tables-processes} - -Esta tabla del sistema se utiliza para implementar el `SHOW PROCESSLIST` consulta. - -Columna: - -- `user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` usuario. El campo contiene el nombre de usuario para una consulta específica, no para una consulta que esta consulta inició. -- `address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` en el servidor de solicitud de consulta. -- `elapsed` (Float64) – The time in seconds since request execution started. -- `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -- `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -- `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. -- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [Método de codificación de datos:](../operations/settings/query-complexity.md#settings_max_memory_usage) configuración. -- `query` (String) – The query text. For `INSERT`, no incluye los datos para insertar. -- `query_id` (String) – Query ID, if defined. - -## sistema.text_log {#system_tables-text_log} - -Contiene entradas de registro. El nivel de registro que va a esta tabla se puede limitar con `text_log.level` configuración del servidor. - -Columna: - -- `event_date` (`Date`) - Fecha de la entrada. -- `event_time` (`DateTime`) - Hora de la entrada. -- `microseconds` (`UInt32`) - Microsegundos de la entrada. -- `thread_name` (String) — Name of the thread from which the logging was done. -- `thread_id` (UInt64) — OS thread ID. -- `level` (`Enum8`) - Nivel de entrada. - - `'Fatal' = 1` - - `'Critical' = 2` - - `'Error' = 3` - - `'Warning' = 4` - - `'Notice' = 5` - - `'Information' = 6` - - `'Debug' = 7` - - `'Trace' = 8` -- `query_id` (`String`) - ID de la consulta. -- `logger_name` (`LowCardinality(String)`) - Name of the logger (i.e. `DDLWorker`) -- `message` (`String`) - El mensaje en sí. -- `revision` (`UInt32`) - Revisión de ClickHouse. -- `source_file` (`LowCardinality(String)`) - Archivo de origen desde el que se realizó el registro. -- `source_line` (`UInt64`) - Línea de origen desde la que se realizó el registro. - -## sistema.query_log {#system_tables-query_log} - -Contiene información sobre la ejecución de consultas. Para cada consulta, puede ver la hora de inicio del procesamiento, la duración del procesamiento, los mensajes de error y otra información. - -!!! note "Nota" - La tabla no contiene datos de entrada para `INSERT` consulta. - -ClickHouse crea esta tabla sólo si el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) se especifica el parámetro server. Este parámetro establece las reglas de registro, como el intervalo de registro o el nombre de la tabla en la que se registrarán las consultas. - -Para habilitar el registro de consultas, [Log_queries](settings/settings.md#settings-log-queries) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. - -El `system.query_log` tabla registra dos tipos de consultas: - -1. Consultas iniciales ejecutadas directamente por el cliente. -2. Consultas secundarias iniciadas por otras consultas (para la ejecución de consultas distribuidas). Para estos tipos de consultas, la información sobre las consultas principales se muestra en el `initial_*` columna. - -Columna: - -- `type` (`Enum8`) — Type of event that occurred when executing the query. Values: - - `'QueryStart' = 1` — Successful start of query execution. - - `'QueryFinish' = 2` — Successful end of query execution. - - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. - - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. -- `event_date` (Date) — Query starting date. -- `event_time` (DateTime) — Query starting time. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` consultas, el número de filas escritas. Para otras consultas, el valor de la columna es 0. -- `written_bytes` (UInt64) — For `INSERT` consultas, el número de bytes escritos. Para otras consultas, el valor de la columna es 0. -- `result_rows` (UInt64) — Number of rows in the result. -- `result_bytes` (UInt64) — Number of bytes in the result. -- `memory_usage` (UInt64) — Memory consumption by the query. -- `query` (String) — Query string. -- `exception` (String) — Exception message. -- `stack_trace` (String) — Stack trace (a list of methods called before the error occurred). An empty string, if the query is completed successfully. -- `is_initial_query` (UInt8) — Query type. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: - - 1 — TCP. - - 2 — HTTP. -- `os_user` (String) — OS's username who runs [Casa de clics-cliente](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [Casa de clics-cliente](../interfaces/cli.md) o se ejecuta otro cliente TCP. -- `client_name` (String) — The [Casa de clics-cliente](../interfaces/cli.md) o otro nombre de cliente TCP. -- `client_revision` (UInt32) — Revision of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_major` (UInt32) — Major version of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_minor` (UInt32) — Minor version of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_patch` (UInt32) — Patch component of the [Casa de clics-cliente](../interfaces/cli.md) o otra versión de cliente TCP. -- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. - - 1 — `GET` se utilizó el método. - - 2 — `POST` se utilizó el método. -- `http_user_agent` (String) — The `UserAgent` encabezado pasado en la solicitud HTTP. -- `quota_key` (String) — The “quota key” especificado en el [cuota](quotas.md) ajuste (ver `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [sistema.evento](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` columna. -- `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parámetro a 1. -- `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` columna. - -Cada consulta crea una o dos filas en el `query_log` tabla, dependiendo del estado de la consulta: - -1. Si la ejecución de la consulta se realiza correctamente, se crean dos eventos con los tipos 1 y 2 (consulte `type` columna). -2. Si se produjo un error durante el procesamiento de la consulta, se crean dos eventos con los tipos 1 y 4. -3. Si se produjo un error antes de iniciar la consulta, se crea un solo evento con el tipo 3. - -De forma predeterminada, los registros se agregan a la tabla a intervalos de 7,5 segundos. Puede establecer este intervalo en el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `flush_interval_milliseconds` parámetro). Para vaciar los registros a la fuerza desde el búfer de memoria a la tabla, utilice `SYSTEM FLUSH LOGS` consulta. - -Cuando la tabla se elimina manualmente, se creará automáticamente sobre la marcha. Tenga en cuenta que se eliminarán todos los registros anteriores. - -!!! note "Nota" - El período de almacenamiento para los registros es ilimitado. Los registros no se eliminan automáticamente de la tabla. Debe organizar la eliminación de registros obsoletos usted mismo. - -Puede especificar una clave de partición arbitraria `system.query_log` mesa en el [query_log](server-configuration-parameters/settings.md#server_configuration_parameters-query-log) configuración del servidor (consulte el `partition_by` parámetro). - -## sistema.Sistema abierto {#system_tables-query-thread-log} - -La tabla contiene información sobre cada subproceso de ejecución de consultas. - -ClickHouse crea esta tabla sólo si el [Sistema abierto.](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) se especifica el parámetro server. Este parámetro establece las reglas de registro, como el intervalo de registro o el nombre de la tabla en la que se registrarán las consultas. - -Para habilitar el registro de consultas, [Log_query_threads](settings/settings.md#settings-log-query-threads) parámetro a 1. Para obtener más información, consulte el [Configuración](settings/settings.md) apartado. - -Columna: - -- `event_date` (Date) — the date when the thread has finished execution of the query. -- `event_time` (DateTime) — the date and time when the thread has finished execution of the query. -- `query_start_time` (DateTime) — Start time of query execution. -- `query_duration_ms` (UInt64) — Duration of query execution. -- `read_rows` (UInt64) — Number of read rows. -- `read_bytes` (UInt64) — Number of read bytes. -- `written_rows` (UInt64) — For `INSERT` consultas, el número de filas escritas. Para otras consultas, el valor de la columna es 0. -- `written_bytes` (UInt64) — For `INSERT` consultas, el número de bytes escritos. Para otras consultas, el valor de la columna es 0. -- `memory_usage` (Int64) — The difference between the amount of allocated and freed memory in context of this thread. -- `peak_memory_usage` (Int64) — The maximum difference between the amount of allocated and freed memory in context of this thread. -- `thread_name` (String) — Name of the thread. -- `thread_number` (UInt32) — Internal thread ID. -- `os_thread_id` (Int32) — OS thread ID. -- `master_thread_id` (UInt64) — OS initial ID of initial thread. -- `query` (String) — Query string. -- `is_initial_query` (UInt8) — Query type. Possible values: - - 1 — Query was initiated by the client. - - 0 — Query was initiated by another query for distributed query execution. -- `user` (String) — Name of the user who initiated the current query. -- `query_id` (String) — ID of the query. -- `address` (IPv6) — IP address that was used to make the query. -- `port` (UInt16) — The client port that was used to make the query. -- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). -- `initial_query_id` (String) — ID of the initial query (for distributed query execution). -- `initial_address` (IPv6) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The client port that was used to make the parent query. -- `interface` (UInt8) — Interface that the query was initiated from. Possible values: - - 1 — TCP. - - 2 — HTTP. -- `os_user` (String) — OS's username who runs [Casa de clics-cliente](../interfaces/cli.md). -- `client_hostname` (String) — Hostname of the client machine where the [Casa de clics-cliente](../interfaces/cli.md) o se ejecuta otro cliente TCP. -- `client_name` (String) — The [Casa de clics-cliente](../interfaces/cli.md) o otro nombre de cliente TCP. -- `client_revision` (UInt32) — Revision of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_major` (UInt32) — Major version of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_minor` (UInt32) — Minor version of the [Casa de clics-cliente](../interfaces/cli.md) o otro cliente TCP. -- `client_version_patch` (UInt32) — Patch component of the [Casa de clics-cliente](../interfaces/cli.md) o otra versión de cliente TCP. -- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - - 0 — The query was launched from the TCP interface. - - 1 — `GET` se utilizó el método. - - 2 — `POST` se utilizó el método. -- `http_user_agent` (String) — The `UserAgent` encabezado pasado en la solicitud HTTP. -- `quota_key` (String) — The “quota key” especificado en el [cuota](quotas.md) ajuste (ver `keyed`). -- `revision` (UInt32) — ClickHouse revision. -- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [sistema.evento](#system_tables-events) -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` columna. - -De forma predeterminada, los registros se agregan a la tabla a intervalos de 7,5 segundos. Puede establecer este intervalo en el [Sistema abierto.](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuración del servidor (consulte el `flush_interval_milliseconds` parámetro). Para vaciar los registros a la fuerza desde el búfer de memoria a la tabla, utilice `SYSTEM FLUSH LOGS` consulta. - -Cuando la tabla se elimina manualmente, se creará automáticamente sobre la marcha. Tenga en cuenta que se eliminarán todos los registros anteriores. - -!!! note "Nota" - El período de almacenamiento para los registros es ilimitado. Los registros no se eliminan automáticamente de la tabla. Debe organizar la eliminación de registros obsoletos usted mismo. - -Puede especificar una clave de partición arbitraria `system.query_thread_log` mesa en el [Sistema abierto.](server-configuration-parameters/settings.md#server_configuration_parameters-query-thread-log) configuración del servidor (consulte el `partition_by` parámetro). - -## sistema.trace_log {#system_tables-trace_log} - -Contiene seguimientos de pila recopilados por el generador de perfiles de consultas de muestreo. - -ClickHouse crea esta tabla cuando el [trace_log](server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) se establece la sección de configuración del servidor. También el [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) y [Los resultados de la prueba](settings/settings.md#query_profiler_cpu_time_period_ns) los ajustes deben establecerse. - -Para analizar los registros, utilice el `addressToLine`, `addressToSymbol` y `demangle` funciones de inspección. - -Columna: - -- `event_date` ([Fecha](../sql-reference/data-types/date.md)) — Date of sampling moment. - -- `event_time` ([FechaHora](../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment. - -- `timestamp_ns` ([UInt64](../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds. - -- `revision` ([UInt32](../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision. - - Cuando se conecta al servidor por `clickhouse-client`, ves la cadena similar a `Connected to ClickHouse server version 19.18.1 revision 54429.`. Este campo contiene el `revision`, pero no el `version` de un servidor. - -- `timer_type` ([Enum8](../sql-reference/data-types/enum.md)) — Timer type: - - - `Real` representa el tiempo del reloj de pared. - - `CPU` representa el tiempo de CPU. - -- `thread_number` ([UInt32](../sql-reference/data-types/int-uint.md)) — Thread identifier. - -- `query_id` ([Cadena](../sql-reference/data-types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) tabla del sistema. - -- `trace` ([Matriz (UInt64)](../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. - -**Ejemplo** - -``` sql -SELECT * FROM system.trace_log LIMIT 1 \G -``` - -``` text -Row 1: -────── -event_date: 2019-11-15 -event_time: 2019-11-15 15:09:38 -revision: 54428 -timer_type: Real -thread_number: 48 -query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915 -trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935] -``` - -## sistema.Replica {#system_tables-replicas} - -Contiene información y estado de las tablas replicadas que residen en el servidor local. -Esta tabla se puede utilizar para el monitoreo. La tabla contiene una fila para cada tabla Replicated\*. - -Ejemplo: - -``` sql -SELECT * -FROM system.replicas -WHERE table = 'visits' -FORMAT Vertical -``` - -``` text -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 -``` - -Columna: - -- `database` (`String`) - Nombre de la base de datos -- `table` (`String`) - Nombre de la tabla -- `engine` (`String`) - Nombre del motor de tabla -- `is_leader` (`UInt8`) - Si la réplica es la líder. - Sólo una réplica a la vez puede ser el líder. El líder es responsable de seleccionar las fusiones de fondo para realizar. - Tenga en cuenta que las escrituras se pueden realizar en cualquier réplica que esté disponible y tenga una sesión en ZK, independientemente de si es un líder. -- `can_become_leader` (`UInt8`) - Si la réplica puede ser elegida como líder. -- `is_readonly` (`UInt8`) - Si la réplica está en modo de sólo lectura. - Este modo se activa si la configuración no tiene secciones con ZooKeeper, si se produce un error desconocido al reinicializar sesiones en ZooKeeper y durante la reinicialización de sesiones en ZooKeeper. -- `is_session_expired` (`UInt8`) - la sesión con ZooKeeper ha expirado. Básicamente lo mismo que `is_readonly`. -- `future_parts` (`UInt32`) - El número de partes de datos que aparecerán como resultado de INSERTs o fusiones que aún no se han realizado. -- `parts_to_check` (`UInt32`) - El número de partes de datos en la cola para la verificación. Una pieza se coloca en la cola de verificación si existe la sospecha de que podría estar dañada. -- `zookeeper_path` (`String`) - Ruta de acceso a los datos de la tabla en ZooKeeper. -- `replica_name` (`String`) - Nombre de réplica en ZooKeeper. Diferentes réplicas de la misma tabla tienen diferentes nombres. -- `replica_path` (`String`) - Ruta de acceso a los datos de réplica en ZooKeeper. Lo mismo que concatenar ‘zookeeper_path/replicas/replica_path’. -- `columns_version` (`Int32`) - Número de versión de la estructura de la tabla. Indica cuántas veces se realizó ALTER. Si las réplicas tienen versiones diferentes, significa que algunas réplicas aún no han hecho todas las ALTER. -- `queue_size` (`UInt32`) - Tamaño de la cola para las operaciones en espera de ser realizadas. Las operaciones incluyen insertar bloques de datos, fusiones y otras acciones. Por lo general, coincide con `future_parts`. -- `inserts_in_queue` (`UInt32`) - Número de inserciones de bloques de datos que deben realizarse. Las inserciones generalmente se replican con bastante rapidez. Si este número es grande, significa que algo anda mal. -- `merges_in_queue` (`UInt32`) - El número de fusiones en espera de hacerse. A veces las fusiones son largas, por lo que este valor puede ser mayor que cero durante mucho tiempo. -- `part_mutations_in_queue` (`UInt32`) - El número de mutaciones a la espera de hacerse. -- `queue_oldest_time` (`DateTime`) - Si `queue_size` mayor que 0, muestra cuándo se agregó la operación más antigua a la cola. -- `inserts_oldest_time` (`DateTime`) - Ver `queue_oldest_time` -- `merges_oldest_time` (`DateTime`) - Ver `queue_oldest_time` -- `part_mutations_oldest_time` (`DateTime`) - Ver `queue_oldest_time` - -Las siguientes 4 columnas tienen un valor distinto de cero solo cuando hay una sesión activa con ZK. - -- `log_max_index` (`UInt64`) - Número máximo de inscripción en el registro de actividad general. -- `log_pointer` (`UInt64`) - Número máximo de entrada en el registro de actividad general que la réplica copió en su cola de ejecución, más uno. Si `log_pointer` es mucho más pequeño que `log_max_index`, algo está mal. -- `last_queue_update` (`DateTime`) - Cuando la cola se actualizó la última vez. -- `absolute_delay` (`UInt64`) - ¿Qué tan grande retraso en segundos tiene la réplica actual. -- `total_replicas` (`UInt8`) - El número total de réplicas conocidas de esta tabla. -- `active_replicas` (`UInt8`) - El número de réplicas de esta tabla que tienen una sesión en ZooKeeper (es decir, el número de réplicas en funcionamiento). - -Si solicita todas las columnas, la tabla puede funcionar un poco lentamente, ya que se realizan varias lecturas de ZooKeeper para cada fila. -Si no solicita las últimas 4 columnas (log_max_index, log_pointer, total_replicas, active_replicas), la tabla funciona rápidamente. - -Por ejemplo, puede verificar que todo funcione correctamente de esta manera: - -``` sql -SELECT - database, - table, - is_leader, - is_readonly, - is_session_expired, - future_parts, - parts_to_check, - columns_version, - queue_size, - inserts_in_queue, - merges_in_queue, - log_max_index, - log_pointer, - total_replicas, - active_replicas -FROM system.replicas -WHERE - is_readonly - OR is_session_expired - OR future_parts > 20 - OR parts_to_check > 10 - OR queue_size > 20 - OR inserts_in_queue > 10 - OR log_max_index - log_pointer > 10 - OR total_replicas < 2 - OR active_replicas < total_replicas -``` - -Si esta consulta no devuelve nada, significa que todo está bien. - -## sistema.configuración {#system-tables-system-settings} - -Contiene información sobre la configuración de sesión para el usuario actual. - -Columna: - -- `name` ([Cadena](../sql-reference/data-types/string.md)) — Setting name. -- `value` ([Cadena](../sql-reference/data-types/string.md)) — Setting value. -- `changed` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether a setting is changed from its default value. -- `description` ([Cadena](../sql-reference/data-types/string.md)) — Short setting description. -- `min` ([NULL](../sql-reference/data-types/nullable.md)([Cadena](../sql-reference/data-types/string.md))) — Minimum value of the setting, if any is set via [limitación](settings/constraints-on-settings.md#constraints-on-settings). Si la configuración no tiene ningún valor mínimo, contiene [NULL](../sql-reference/syntax.md#null-literal). -- `max` ([NULL](../sql-reference/data-types/nullable.md)([Cadena](../sql-reference/data-types/string.md))) — Maximum value of the setting, if any is set via [limitación](settings/constraints-on-settings.md#constraints-on-settings). Si la configuración no tiene ningún valor máximo, contiene [NULL](../sql-reference/syntax.md#null-literal). -- `readonly` ([UInt8](../sql-reference/data-types/int-uint.md#uint-ranges)) — Shows whether the current user can change the setting: - - `0` — Current user can change the setting. - - `1` — Current user can't change the setting. - -**Ejemplo** - -En el ejemplo siguiente se muestra cómo obtener información sobre la configuración cuyo nombre contiene `min_i`. - -``` sql -SELECT * -FROM system.settings -WHERE name LIKE '%min_i%' -``` - -``` text -┌─name────────────────────────────────────────┬─value─────┬─changed─┬─description───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┬─min──┬─max──┬─readonly─┐ -│ min_insert_block_size_rows │ 1048576 │ 0 │ Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ min_insert_block_size_bytes │ 268435456 │ 0 │ Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -│ read_backoff_min_interval_between_events_ms │ 1000 │ 0 │ Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time. │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ -└─────────────────────────────────────────────┴───────────┴─────────┴───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┴──────┴──────┴──────────┘ -``` - -Uso de `WHERE changed` puede ser útil, por ejemplo, cuando se desea comprobar: - -- Si los ajustes de los archivos de configuración se cargan correctamente y están en uso. -- Configuración que cambió en la sesión actual. - - - -``` sql -SELECT * FROM system.settings WHERE changed AND name='load_balancing' -``` - -**Ver también** - -- [Configuración](settings/index.md#session-settings-intro) -- [Permisos para consultas](settings/permissions-for-queries.md#settings_readonly) -- [Restricciones en la configuración](settings/constraints-on-settings.md) - -## sistema.table_engines {#system.table_engines} - -``` text -┌─name───────────────────┬─value───────┐ -│ max_threads │ 8 │ -│ use_uncompressed_cache │ 0 │ -│ load_balancing │ random │ -│ max_memory_usage │ 10000000000 │ -└────────────────────────┴─────────────┘ -``` - -## sistema.merge_tree_settings {#system-merge_tree_settings} - -Contiene información sobre la configuración `MergeTree` tabla. - -Columna: - -- `name` (String) — Setting name. -- `value` (String) — Setting value. -- `description` (String) — Setting description. -- `type` (String) — Setting type (implementation specific string value). -- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. - -## sistema.table_engines {#system-table-engines} - -Contiene la descripción de los motores de tablas admitidos por el servidor y su información de soporte de características. - -Esta tabla contiene las siguientes columnas (el tipo de columna se muestra entre corchetes): - -- `name` (String) — The name of table engine. -- `supports_settings` (UInt8) — Flag that indicates if table engine supports `SETTINGS` clausula. -- `supports_skipping_indices` (UInt8) — Flag that indicates if table engine supports [Índices de saltos](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes). -- `supports_ttl` (UInt8) — Flag that indicates if table engine supports [TTL](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). -- `supports_sort_order` (UInt8) — Flag that indicates if table engine supports clauses `PARTITION_BY`, `PRIMARY_KEY`, `ORDER_BY` y `SAMPLE_BY`. -- `supports_replication` (UInt8) — Flag that indicates if table engine supports [Replicación de datos](../engines/table-engines/mergetree-family/replication.md). -- `supports_duduplication` (UInt8) — Flag that indicates if table engine supports data deduplication. - -Ejemplo: - -``` sql -SELECT * -FROM system.table_engines -WHERE name in ('Kafka', 'MergeTree', 'ReplicatedCollapsingMergeTree') -``` - -``` text -┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┐ -│ Kafka │ 1 │ 0 │ 0 │ 0 │ 0 │ 0 │ -│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ -│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -└───────────────────────────────┴───────────────────┴───────────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┘ -``` - -**Ver también** - -- Familia MergeTree [cláusulas de consulta](../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) -- Kafka [configuración](../engines/table-engines/integrations/kafka.md#table_engine-kafka-creating-a-table) -- Unir [configuración](../engines/table-engines/special/join.md#join-limitations-and-settings) - -## sistema.tabla {#system-tables} - -Contiene metadatos de cada tabla que el servidor conoce. Las tablas separadas no se muestran en `system.tables`. - -Esta tabla contiene las siguientes columnas (el tipo de columna se muestra entre corchetes): - -- `database` (String) — The name of the database the table is in. - -- `name` (String) — Table name. - -- `engine` (String) — Table engine name (without parameters). - -- `is_temporary` (UInt8): marca que indica si la tabla es temporal. - -- `data_path` (String) - Ruta de acceso a los datos de la tabla en el sistema de archivos. - -- `metadata_path` (String) - Ruta de acceso a los metadatos de la tabla en el sistema de archivos. - -- `metadata_modification_time` (DateTime) - Hora de la última modificación de los metadatos de la tabla. - -- `dependencies_database` (Array(String)) - Dependencias de base de datos. - -- `dependencies_table` (Array(String)) - Dependencias de tabla ([Método de codificación de datos:](../engines/table-engines/special/materializedview.md) tablas basadas en la tabla actual). - -- `create_table_query` (String) - La consulta que se utilizó para crear la tabla. - -- `engine_full` (String) - Parámetros del motor de tabla. - -- `partition_key` (String) - La expresión de clave de partición especificada en la tabla. - -- `sorting_key` (String) - La expresión de clave de ordenación especificada en la tabla. - -- `primary_key` (String) - La expresión de clave principal especificada en la tabla. - -- `sampling_key` (String) - La expresión de clave de muestreo especificada en la tabla. - -- `storage_policy` (String) - La política de almacenamiento: - - - [Método de codificación de datos:](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - - [Distribuido](../engines/table-engines/special/distributed.md#distributed) - -- `total_rows` (Nullable(UInt64)) - Número total de filas, si es posible determinar rápidamente el número exacto de filas en la tabla, de lo contrario `Null` (incluyendo underying `Buffer` tabla). - -- `total_bytes` (Nullable(UInt64)) - Número total de bytes, si es posible determinar rápidamente el número exacto de bytes para la tabla en el almacenamiento, de lo contrario `Null` (**no** incluye cualquier almacenamiento subyacente). - - - If the table stores data on disk, returns used space on disk (i.e. compressed). - - Si la tabla almacena datos en la memoria, devuelve el número aproximado de bytes utilizados en la memoria. - -El `system.tables` se utiliza en `SHOW TABLES` implementación de consultas. - -## sistema.Zookeeper {#system-zookeeper} - -La tabla no existe si ZooKeeper no está configurado. Permite leer datos del clúster ZooKeeper definido en la configuración. -La consulta debe tener un ‘path’ condición de igualdad en la cláusula WHERE. Este es el camino en ZooKeeper para los niños para los que desea obtener datos. - -Consulta `SELECT * FROM system.zookeeper WHERE path = '/clickhouse'` salidas de datos para todos los niños en el `/clickhouse` nodo. -Para generar datos para todos los nodos raíz, escriba path = ‘/’. -Si la ruta especificada en ‘path’ no existe, se lanzará una excepción. - -Columna: - -- `name` (String) — The name of the node. -- `path` (String) — The path to the node. -- `value` (String) — Node value. -- `dataLength` (Int32) — Size of the value. -- `numChildren` (Int32) — Number of descendants. -- `czxid` (Int64) — ID of the transaction that created the node. -- `mzxid` (Int64) — ID of the transaction that last changed the node. -- `pzxid` (Int64) — ID of the transaction that last deleted or added descendants. -- `ctime` (DateTime) — Time of node creation. -- `mtime` (DateTime) — Time of the last modification of the node. -- `version` (Int32) — Node version: the number of times the node was changed. -- `cversion` (Int32) — Number of added or removed descendants. -- `aversion` (Int32) — Number of changes to the ACL. -- `ephemeralOwner` (Int64) — For ephemeral nodes, the ID of the session that owns this node. - -Ejemplo: - -``` sql -SELECT * -FROM system.zookeeper -WHERE path = '/clickhouse/tables/01-08/visits/replicas' -FORMAT Vertical -``` - -``` text -Row 1: -────── -name: example01-08-1.yandex.ru -value: -czxid: 932998691229 -mzxid: 932998691229 -ctime: 2015-03-27 16:49:51 -mtime: 2015-03-27 16:49:51 -version: 0 -cversion: 47 -aversion: 0 -ephemeralOwner: 0 -dataLength: 0 -numChildren: 7 -pzxid: 987021031383 -path: /clickhouse/tables/01-08/visits/replicas - -Row 2: -────── -name: example01-08-2.yandex.ru -value: -czxid: 933002738135 -mzxid: 933002738135 -ctime: 2015-03-27 16:57:01 -mtime: 2015-03-27 16:57:01 -version: 0 -cversion: 37 -aversion: 0 -ephemeralOwner: 0 -dataLength: 0 -numChildren: 7 -pzxid: 987021252247 -path: /clickhouse/tables/01-08/visits/replicas -``` - -## sistema.mutación {#system_tables-mutations} - -La tabla contiene información sobre [mutación](../sql-reference/statements/alter.md#alter-mutations) de las tablas MergeTree y su progreso. Cada comando de mutación está representado por una sola fila. La tabla tiene las siguientes columnas: - -**base**, **tabla** - El nombre de la base de datos y la tabla a la que se aplicó la mutación. - -**mutation_id** - La identificación de la mutación. Para las tablas replicadas, estos identificadores corresponden a los nombres de znode `/mutations/` directorio en ZooKeeper. Para las tablas no duplicadas, los ID corresponden a los nombres de archivo en el directorio de datos de la tabla. - -**comando** - La cadena de comandos de mutación (la parte de la consulta después de `ALTER TABLE [db.]table`). - -**create_time** - Cuando este comando de mutación fue enviado para su ejecución. - -**block_numbers.partition_id**, **block_numbers.numero** - Una columna anidada. Para las mutaciones de tablas replicadas, contiene un registro para cada partición: el ID de partición y el número de bloque que fue adquirido por la mutación (en cada partición, solo se mutarán las partes que contienen bloques con números menores que el número de bloque adquirido por la mutación en esa partición). En tablas no replicadas, los números de bloque en todas las particiones forman una sola secuencia. Esto significa que para las mutaciones de tablas no replicadas, la columna contendrá un registro con un solo número de bloque adquirido por la mutación. - -**partes_a_do** - El número de partes de datos que deben mutarse para que finalice la mutación. - -**is_done** - Es la mutación hecho? Tenga en cuenta que incluso si `parts_to_do = 0` es posible que aún no se haya realizado una mutación de una tabla replicada debido a un INSERT de larga ejecución que creará una nueva parte de datos que deberá mutarse. - -Si hubo problemas con la mutación de algunas partes, las siguientes columnas contienen información adicional: - -**Método de codificación de datos:** - El nombre de la parte más reciente que no se pudo mutar. - -**Método de codificación de datos:** - El momento del fracaso de la mutación de la parte más reciente. - -**Método de codificación de datos:** - El mensaje de excepción que causó el error de mutación de parte más reciente. - -## sistema.disco {#system_tables-disks} - -Contiene información sobre los discos definidos en el [configuración del servidor](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). - -Columna: - -- `name` ([Cadena](../sql-reference/data-types/string.md)) — Name of a disk in the server configuration. -- `path` ([Cadena](../sql-reference/data-types/string.md)) — Path to the mount point in the file system. -- `free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Free space on disk in bytes. -- `total_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Disk volume in bytes. -- `keep_free_space` ([UInt64](../sql-reference/data-types/int-uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parámetro de configuración del disco. - -## sistema.almacenamiento_policies {#system_tables-storage_policies} - -Contiene información sobre las directivas de almacenamiento y los volúmenes [configuración del servidor](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes_configure). - -Columna: - -- `policy_name` ([Cadena](../sql-reference/data-types/string.md)) — Name of the storage policy. -- `volume_name` ([Cadena](../sql-reference/data-types/string.md)) — Volume name defined in the storage policy. -- `volume_priority` ([UInt64](../sql-reference/data-types/int-uint.md)) — Volume order number in the configuration. -- `disks` ([Array(Cadena)](../sql-reference/data-types/array.md)) — Disk names, defined in the storage policy. -- `max_data_part_size` ([UInt64](../sql-reference/data-types/int-uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). -- `move_factor` ([Float64](../sql-reference/data-types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. - -Si la directiva de almacenamiento contiene más de un volumen, la información de cada volumen se almacena en la fila individual de la tabla. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/system_tables/) diff --git a/docs/es/operations/tips.md b/docs/es/operations/tips.md deleted file mode 100644 index deb226450aa..00000000000 --- a/docs/es/operations/tips.md +++ /dev/null @@ -1,251 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: Recomendaciones de uso ---- - -# Recomendaciones de uso {#usage-recommendations} - -## CPU Scaling Governor {#cpu-scaling-governor} - -Utilice siempre el `performance` gobernador de escala. El `on-demand` regulador de escala funciona mucho peor con una demanda constante. - -``` bash -$ echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor -``` - -## Limitaciones de la CPU {#cpu-limitations} - -Los procesadores pueden sobrecalentarse. Utilizar `dmesg` para ver si la velocidad de reloj de la CPU era limitada debido al sobrecalentamiento. -La restricción también se puede establecer externamente en el nivel del centro de datos. Usted puede utilizar `turbostat` para controlarlo bajo una carga. - -## RAM {#ram} - -Para pequeñas cantidades de datos (hasta ~200 GB comprimidos), es mejor usar tanta memoria como el volumen de datos. -Para grandes cantidades de datos y al procesar consultas interactivas (en línea), debe usar una cantidad razonable de RAM (128 GB o más) para que el subconjunto de datos en caliente quepa en la memoria caché de páginas. -Incluso para volúmenes de datos de ~ 50 TB por servidor, el uso de 128 GB de RAM mejora significativamente el rendimiento de las consultas en comparación con 64 GB. - -No deshabilite el sobrecompromiso. Valor `cat /proc/sys/vm/overcommit_memory` debe ser 0 o 1. Ejecutar - -``` bash -$ echo 0 | sudo tee /proc/sys/vm/overcommit_memory -``` - -## Páginas enormes {#huge-pages} - -Siempre deshabilite las páginas enormes transparentes. Interfiere con los asignadores de memoria, lo que conduce a una degradación significativa del rendimiento. - -``` bash -$ echo 'never' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled -``` - -Utilizar `perf top` para ver el tiempo pasado en el kernel para la administración de memoria. -Las páginas enormes permanentes tampoco necesitan ser asignadas. - -## Subsistema de almacenamiento {#storage-subsystem} - -Si su presupuesto le permite usar SSD, use SSD. -Si no, use HDD. Los discos duros SATA 7200 RPM servirán. - -Dar preferencia a una gran cantidad de servidores con discos duros locales sobre un número menor de servidores con estantes de discos conectados. -Pero para almacenar archivos con consultas raras, los estantes funcionarán. - -## RAID {#raid} - -Al usar HDD, puede combinar su RAID-10, RAID-5, RAID-6 o RAID-50. -Para Linux, el software RAID es mejor (con `mdadm`). No recomendamos usar LVM. -Al crear RAID-10, seleccione el `far` diseño. -Si su presupuesto lo permite, elija RAID-10. - -Si tiene más de 4 discos, utilice RAID-6 (preferido) o RAID-50, en lugar de RAID-5. -Cuando use RAID-5, RAID-6 o RAID-50, siempre aumente stripe_cache_size, ya que el valor predeterminado generalmente no es la mejor opción. - -``` bash -$ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size -``` - -Calcule el número exacto a partir del número de dispositivos y el tamaño del bloque, utilizando la fórmula: `2 * num_devices * chunk_size_in_bytes / 4096`. - -Un tamaño de bloque de 1024 KB es suficiente para todas las configuraciones RAID. -Nunca ajuste el tamaño del bloque demasiado pequeño o demasiado grande. - -Puede usar RAID-0 en SSD. -Independientemente del uso de RAID, utilice siempre la replicación para la seguridad de los datos. - -Habilite NCQ con una cola larga. Para HDD, elija el programador CFQ, y para SSD, elija noop. No reduzca el ‘readahead’ configuración. -Para HDD, habilite la memoria caché de escritura. - -## Sistema de archivos {#file-system} - -Ext4 es la opción más confiable. Establecer las opciones de montaje `noatime, nobarrier`. -XFS también es adecuado, pero no ha sido probado tan a fondo con ClickHouse. -La mayoría de los otros sistemas de archivos también deberían funcionar bien. Los sistemas de archivos con asignación retrasada funcionan mejor. - -## Núcleo de Linux {#linux-kernel} - -No use un kernel de Linux obsoleto. - -## Red {#network} - -Si está utilizando IPv6, aumente el tamaño de la caché de ruta. -El kernel de Linux anterior a 3.2 tenía una multitud de problemas con la implementación de IPv6. - -Utilice al menos una red de 10 GB, si es posible. 1 Gb también funcionará, pero será mucho peor para parchear réplicas con decenas de terabytes de datos, o para procesar consultas distribuidas con una gran cantidad de datos intermedios. - -## ZooKeeper {#zookeeper} - -Probablemente ya esté utilizando ZooKeeper para otros fines. Puede usar la misma instalación de ZooKeeper, si aún no está sobrecargada. - -It's best to use a fresh version of ZooKeeper – 3.4.9 or later. The version in stable Linux distributions may be outdated. - -Nunca debe usar scripts escritos manualmente para transferir datos entre diferentes clústeres de ZooKeeper, ya que el resultado será incorrecto para los nodos secuenciales. Nunca utilice el “zkcopy” utilidad por la misma razón: https://github.com/ksprojects/zkcopy/issues/15 - -Si desea dividir un clúster ZooKeeper existente en dos, la forma correcta es aumentar el número de sus réplicas y, a continuación, volver a configurarlo como dos clústeres independientes. - -No ejecute ZooKeeper en los mismos servidores que ClickHouse. Porque ZooKeeper es muy sensible a la latencia y ClickHouse puede utilizar todos los recursos del sistema disponibles. - -Con la configuración predeterminada, ZooKeeper es una bomba de tiempo: - -> El servidor ZooKeeper no eliminará archivos de instantáneas y registros antiguos cuando utilice la configuración predeterminada (consulte autopurge), y esto es responsabilidad del operador. - -Esta bomba debe ser desactivada. - -La configuración ZooKeeper (3.5.1) a continuación se usa en Yandex.Entorno de producción de Métrica al 20 de mayo de 2017: - -zoológico.Cómo: - -``` bash -# http://hadoop.apache.org/zookeeper/docs/current/zookeeperAdmin.html - -# The number of milliseconds of each tick -tickTime=2000 -# The number of ticks that the initial -# synchronization phase can take -initLimit=30000 -# The number of ticks that can pass between -# sending a request and getting an acknowledgement -syncLimit=10 - -maxClientCnxns=2000 - -maxSessionTimeout=60000000 -# the directory where the snapshot is stored. -dataDir=/opt/zookeeper/{{ '{{' }} cluster['name'] {{ '}}' }}/data -# Place the dataLogDir to a separate physical disc for better performance -dataLogDir=/opt/zookeeper/{{ '{{' }} cluster['name'] {{ '}}' }}/logs - -autopurge.snapRetainCount=10 -autopurge.purgeInterval=1 - - -# To avoid seeks ZooKeeper allocates space in the transaction log file in -# blocks of preAllocSize kilobytes. The default block size is 64M. One reason -# for changing the size of the blocks is to reduce the block size if snapshots -# are taken more often. (Also, see snapCount). -preAllocSize=131072 - -# Clients can submit requests faster than ZooKeeper can process them, -# especially if there are a lot of clients. To prevent ZooKeeper from running -# out of memory due to queued requests, ZooKeeper will throttle clients so that -# there is no more than globalOutstandingLimit outstanding requests in the -# system. The default limit is 1,000.ZooKeeper logs transactions to a -# transaction log. After snapCount transactions are written to a log file a -# snapshot is started and a new transaction log file is started. The default -# snapCount is 10,000. -snapCount=3000000 - -# If this option is defined, requests will be will logged to a trace file named -# traceFile.year.month.day. -#traceFile= - -# Leader accepts client connections. Default value is "yes". The leader machine -# coordinates updates. For higher update throughput at thes slight expense of -# read throughput the leader can be configured to not accept clients and focus -# on coordination. -leaderServes=yes - -standaloneEnabled=false -dynamicConfigFile=/etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/zoo.cfg.dynamic -``` - -Versión Java: - -``` text -Java(TM) SE Runtime Environment (build 1.8.0_25-b17) -Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) -``` - -Parámetros de JVM: - -``` bash -NAME=zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }} -ZOOCFGDIR=/etc/$NAME/conf - -# TODO this is really ugly -# How to find out, which jars are needed? -# seems, that log4j requires the log4j.properties file to be in the classpath -CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper/zookeeper-3.5.1-metrika.jar:/usr/share/zookeeper/slf4j-log4j12-1.7.5.jar:/usr/share/zookeeper/slf4j-api-1.7.5.jar:/usr/share/zookeeper/servlet-api-2.5-20081211.jar:/usr/share/zookeeper/netty-3.7.0.Final.jar:/usr/share/zookeeper/log4j-1.2.16.jar:/usr/share/zookeeper/jline-2.11.jar:/usr/share/zookeeper/jetty-util-6.1.26.jar:/usr/share/zookeeper/jetty-6.1.26.jar:/usr/share/zookeeper/javacc.jar:/usr/share/zookeeper/jackson-mapper-asl-1.9.11.jar:/usr/share/zookeeper/jackson-core-asl-1.9.11.jar:/usr/share/zookeeper/commons-cli-1.2.jar:/usr/src/java/lib/*.jar:/usr/etc/zookeeper" - -ZOOCFG="$ZOOCFGDIR/zoo.cfg" -ZOO_LOG_DIR=/var/log/$NAME -USER=zookeeper -GROUP=zookeeper -PIDDIR=/var/run/$NAME -PIDFILE=$PIDDIR/$NAME.pid -SCRIPTNAME=/etc/init.d/$NAME -JAVA=/usr/bin/java -ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain" -ZOO_LOG4J_PROP="INFO,ROLLINGFILE" -JMXLOCALONLY=false -JAVA_OPTS="-Xms{{ '{{' }} cluster.get('xms','128M') {{ '}}' }} \ - -Xmx{{ '{{' }} cluster.get('xmx','1G') {{ '}}' }} \ - -Xloggc:/var/log/$NAME/zookeeper-gc.log \ - -XX:+UseGCLogFileRotation \ - -XX:NumberOfGCLogFiles=16 \ - -XX:GCLogFileSize=16M \ - -verbose:gc \ - -XX:+PrintGCTimeStamps \ - -XX:+PrintGCDateStamps \ - -XX:+PrintGCDetails - -XX:+PrintTenuringDistribution \ - -XX:+PrintGCApplicationStoppedTime \ - -XX:+PrintGCApplicationConcurrentTime \ - -XX:+PrintSafepointStatistics \ - -XX:+UseParNewGC \ - -XX:+UseConcMarkSweepGC \ --XX:+CMSParallelRemarkEnabled" -``` - -Sal init: - -``` text -description "zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }} centralized coordination service" - -start on runlevel [2345] -stop on runlevel [!2345] - -respawn - -limit nofile 8192 8192 - -pre-start script - [ -r "/etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/environment" ] || exit 0 - . /etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/environment - [ -d $ZOO_LOG_DIR ] || mkdir -p $ZOO_LOG_DIR - chown $USER:$GROUP $ZOO_LOG_DIR -end script - -script - . /etc/zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }}/conf/environment - [ -r /etc/default/zookeeper ] && . /etc/default/zookeeper - if [ -z "$JMXDISABLE" ]; then - JAVA_OPTS="$JAVA_OPTS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY" - fi - exec start-stop-daemon --start -c $USER --exec $JAVA --name zookeeper-{{ '{{' }} cluster['name'] {{ '}}' }} \ - -- -cp $CLASSPATH $JAVA_OPTS -Dzookeeper.log.dir=${ZOO_LOG_DIR} \ - -Dzookeeper.root.logger=${ZOO_LOG4J_PROP} $ZOOMAIN $ZOOCFG -end script -``` - -{## [Artículo Original](https://clickhouse.tech/docs/en/operations/tips/) ##} diff --git a/docs/es/operations/troubleshooting.md b/docs/es/operations/troubleshooting.md deleted file mode 100644 index 9e8d2caca59..00000000000 --- a/docs/es/operations/troubleshooting.md +++ /dev/null @@ -1,146 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 46 -toc_title: "Soluci\xF3n de problemas" ---- - -# Solución de problemas {#troubleshooting} - -- [Instalación](#troubleshooting-installation-errors) -- [Conexión al servidor](#troubleshooting-accepts-no-connections) -- [Procesamiento de consultas](#troubleshooting-does-not-process-queries) -- [Eficiencia del procesamiento de consultas](#troubleshooting-too-slow) - -## Instalación {#troubleshooting-installation-errors} - -### No puede obtener paquetes Deb del repositorio ClickHouse con Apt-get {#you-cannot-get-deb-packages-from-clickhouse-repository-with-apt-get} - -- Compruebe la configuración del firewall. -- Si no puede acceder al repositorio por cualquier motivo, descargue los paquetes como se describe en el [Primeros pasos](../getting-started/index.md) artículo e instálelos manualmente usando el `sudo dpkg -i ` comando. También necesitará el `tzdata` paquete. - -## Conexión al servidor {#troubleshooting-accepts-no-connections} - -Posibles problemas: - -- El servidor no se está ejecutando. -- Parámetros de configuración inesperados o incorrectos. - -### El servidor no se está ejecutando {#server-is-not-running} - -**Compruebe si el servidor está ejecutado** - -Comando: - -``` bash -$ sudo service clickhouse-server status -``` - -Si el servidor no se está ejecutando, inícielo con el comando: - -``` bash -$ sudo service clickhouse-server start -``` - -**Comprobar registros** - -El registro principal de `clickhouse-server` está en `/var/log/clickhouse-server/clickhouse-server.log` predeterminada. - -Si el servidor se inició correctamente, debería ver las cadenas: - -- ` Application: starting up.` — Server started. -- ` Application: Ready for connections.` — Server is running and ready for connections. - -Si `clickhouse-server` error de inicio con un error de configuración, debería ver el `` cadena con una descripción de error. Por ejemplo: - -``` text -2019.01.11 15:23:25.549505 [ 45 ] {} ExternalDictionaries: Failed reloading 'event2id' external dictionary: Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused, e.what() = Connection refused -``` - -Si no ve un error al final del archivo, revise todo el archivo a partir de la cadena: - -``` text - Application: starting up. -``` - -Si intenta iniciar una segunda instancia de `clickhouse-server` en el servidor, verá el siguiente registro: - -``` text -2019.01.11 15:25:11.151730 [ 1 ] {} : Starting ClickHouse 19.1.0 with revision 54413 -2019.01.11 15:25:11.154578 [ 1 ] {} Application: starting up -2019.01.11 15:25:11.156361 [ 1 ] {} StatusFile: Status file ./status already exists - unclean restart. Contents: -PID: 8510 -Started at: 2019-01-11 15:24:23 -Revision: 54413 - -2019.01.11 15:25:11.156673 [ 1 ] {} Application: DB::Exception: Cannot lock file ./status. Another server instance in same directory is already running. -2019.01.11 15:25:11.156682 [ 1 ] {} Application: shutting down -2019.01.11 15:25:11.156686 [ 1 ] {} Application: Uninitializing subsystem: Logging Subsystem -2019.01.11 15:25:11.156716 [ 2 ] {} BaseDaemon: Stop SignalListener thread -``` - -**Ver sistema.d registros** - -Si no encuentra ninguna información útil en `clickhouse-server` registros o no hay registros, puede ver `system.d` registros usando el comando: - -``` bash -$ sudo journalctl -u clickhouse-server -``` - -**Iniciar clickhouse-server en modo interactivo** - -``` bash -$ sudo -u clickhouse /usr/bin/clickhouse-server --config-file /etc/clickhouse-server/config.xml -``` - -Este comando inicia el servidor como una aplicación interactiva con parámetros estándar del script de inicio automático. En este modo `clickhouse-server` imprime todos los mensajes de eventos en la consola. - -### Parámetros de configuración {#configuration-parameters} - -Comprobar: - -- Configuración de Docker. - - Si ejecuta ClickHouse en Docker en una red IPv6, asegúrese de que `network=host` se establece. - -- Configuración del punto final. - - Comprobar [listen_host](server-configuration-parameters/settings.md#server_configuration_parameters-listen_host) y [Tcp_port](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) configuración. - - El servidor ClickHouse acepta conexiones localhost solo de forma predeterminada. - -- Configuración del protocolo HTTP. - - Compruebe la configuración del protocolo para la API HTTP. - -- Configuración de conexión segura. - - Comprobar: - - - El [Tcp_port_secure](server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) configuración. - - Ajustes para [Sertificados SSL](server-configuration-parameters/settings.md#server_configuration_parameters-openssl). - - Utilice los parámetros adecuados mientras se conecta. Por ejemplo, utilice el `port_secure` parámetro con `clickhouse_client`. - -- Configuración del usuario. - - Es posible que esté utilizando el nombre de usuario o la contraseña incorrectos. - -## Procesamiento de consultas {#troubleshooting-does-not-process-queries} - -Si ClickHouse no puede procesar la consulta, envía una descripción de error al cliente. En el `clickhouse-client` obtienes una descripción del error en la consola. Si está utilizando la interfaz HTTP, ClickHouse envía la descripción del error en el cuerpo de la respuesta. Por ejemplo: - -``` bash -$ curl 'http://localhost:8123/' --data-binary "SELECT a" -Code: 47, e.displayText() = DB::Exception: Unknown identifier: a. Note that there are no tables (FROM clause) in your query, context: required_names: 'a' source_tables: table_aliases: private_aliases: column_aliases: public_columns: 'a' masked_columns: array_join_columns: source_columns: , e.what() = DB::Exception -``` - -Si empiezas `clickhouse-client` con el `stack-trace` parámetro, ClickHouse devuelve el seguimiento de la pila del servidor con la descripción de un error. - -Es posible que vea un mensaje sobre una conexión rota. En este caso, puede repetir la consulta. Si la conexión se rompe cada vez que realiza la consulta, compruebe si hay errores en los registros del servidor. - -## Eficiencia del procesamiento de consultas {#troubleshooting-too-slow} - -Si ve que ClickHouse funciona demasiado lentamente, debe perfilar la carga en los recursos del servidor y la red para sus consultas. - -Puede utilizar la utilidad clickhouse-benchmark para crear perfiles de consultas. Muestra el número de consultas procesadas por segundo, el número de filas procesadas por segundo y percentiles de tiempos de procesamiento de consultas. diff --git a/docs/es/operations/update.md b/docs/es/operations/update.md deleted file mode 100644 index 11d15381d72..00000000000 --- a/docs/es/operations/update.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 47 -toc_title: "Actualizaci\xF3n de ClickHouse" ---- - -# Actualización de ClickHouse {#clickhouse-update} - -Si se instaló ClickHouse desde paquetes deb, ejecute los siguientes comandos en el servidor: - -``` bash -$ sudo apt-get update -$ sudo apt-get install clickhouse-client clickhouse-server -$ sudo service clickhouse-server restart -``` - -Si ha instalado ClickHouse utilizando algo distinto de los paquetes deb recomendados, utilice el método de actualización adecuado. - -ClickHouse no admite una actualización distribuida. La operación debe realizarse consecutivamente en cada servidor separado. No actualice todos los servidores de un clúster simultáneamente, o el clúster no estará disponible durante algún tiempo. diff --git a/docs/es/operations/utilities/clickhouse-benchmark.md b/docs/es/operations/utilities/clickhouse-benchmark.md deleted file mode 100644 index 9bcafa40dfe..00000000000 --- a/docs/es/operations/utilities/clickhouse-benchmark.md +++ /dev/null @@ -1,156 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 61 -toc_title: Sistema abierto. ---- - -# Sistema abierto {#clickhouse-benchmark} - -Se conecta a un servidor ClickHouse y envía repetidamente las consultas especificadas. - -Sintaxis: - -``` bash -$ echo "single query" | clickhouse-benchmark [keys] -``` - -o - -``` bash -$ clickhouse-benchmark [keys] <<< "single query" -``` - -Si desea enviar un conjunto de consultas, cree un archivo de texto y coloque cada consulta en la cadena individual de este archivo. Por ejemplo: - -``` sql -SELECT * FROM system.numbers LIMIT 10000000 -SELECT 1 -``` - -Luego pase este archivo a una entrada estándar de `clickhouse-benchmark`. - -``` bash -clickhouse-benchmark [keys] < queries_file -``` - -## Claves {#clickhouse-benchmark-keys} - -- `-c N`, `--concurrency=N` — Number of queries that `clickhouse-benchmark` se envía simultáneamente. Valor predeterminado: 1. -- `-d N`, `--delay=N` — Interval in seconds between intermediate reports (set 0 to disable reports). Default value: 1. -- `-h WORD`, `--host=WORD` — Server host. Default value: `localhost`. Para el [modo de comparación](#clickhouse-benchmark-comparison-mode) puedes usar múltiples `-h` claves. -- `-p N`, `--port=N` — Server port. Default value: 9000. For the [modo de comparación](#clickhouse-benchmark-comparison-mode) puedes usar múltiples `-p` claves. -- `-i N`, `--iterations=N` — Total number of queries. Default value: 0. -- `-r`, `--randomize` — Random order of queries execution if there is more then one input query. -- `-s`, `--secure` — Using TLS connection. -- `-t N`, `--timelimit=N` — Time limit in seconds. `clickhouse-benchmark` detiene el envío de consultas cuando se alcanza el límite de tiempo especificado. Valor predeterminado: 0 (límite de tiempo desactivado). -- `--confidence=N` — Level of confidence for T-test. Possible values: 0 (80%), 1 (90%), 2 (95%), 3 (98%), 4 (99%), 5 (99.5%). Default value: 5. In the [modo de comparación](#clickhouse-benchmark-comparison-mode) `clickhouse-benchmark` realiza el [Prueba t independiente de dos muestras para estudiantes](https://en.wikipedia.org/wiki/Student%27s_t-test#Independent_two-sample_t-test) prueba para determinar si las dos distribuciones no son diferentes con el nivel de confianza seleccionado. -- `--cumulative` — Printing cumulative data instead of data per interval. -- `--database=DATABASE_NAME` — ClickHouse database name. Default value: `default`. -- `--json=FILEPATH` — JSON output. When the key is set, `clickhouse-benchmark` emite un informe al archivo JSON especificado. -- `--user=USERNAME` — ClickHouse user name. Default value: `default`. -- `--password=PSWD` — ClickHouse user password. Default value: empty string. -- `--stacktrace` — Stack traces output. When the key is set, `clickhouse-bencmark` las salidas acumulan rastros de excepciones. -- `--stage=WORD` — Query processing stage at server. ClickHouse stops query processing and returns answer to `clickhouse-benchmark` en la etapa especificada. Valores posibles: `complete`, `fetch_columns`, `with_mergeable_state`. Valor predeterminado: `complete`. -- `--help` — Shows the help message. - -Si desea aplicar alguna [configuración](../../operations/settings/index.md) para consultas, páselas como una clave `--= SETTING_VALUE`. Por ejemplo, `--max_memory_usage=1048576`. - -## Salida {#clickhouse-benchmark-output} - -Predeterminada, `clickhouse-benchmark` informes para cada `--delay` intervalo. - -Ejemplo del informe: - -``` text -Queries executed: 10. - -localhost:9000, queries 10, QPS: 6.772, RPS: 67904487.440, MiB/s: 518.070, result RPS: 67721584.984, result MiB/s: 516.675. - -0.000% 0.145 sec. -10.000% 0.146 sec. -20.000% 0.146 sec. -30.000% 0.146 sec. -40.000% 0.147 sec. -50.000% 0.148 sec. -60.000% 0.148 sec. -70.000% 0.148 sec. -80.000% 0.149 sec. -90.000% 0.150 sec. -95.000% 0.150 sec. -99.000% 0.150 sec. -99.900% 0.150 sec. -99.990% 0.150 sec. -``` - -En el informe puedes encontrar: - -- Número de consultas en el `Queries executed:` campo. - -- Cadena de estado que contiene (en orden): - - - Punto final del servidor ClickHouse. - - Número de consultas procesadas. - - QPS: QPS: ¿Cuántas consultas realizó el servidor por segundo durante un período `--delay` argumento. - - RPS: ¿Cuántas filas lee el servidor por segundo durante un período `--delay` argumento. - - MiB/s: ¿Cuántos mebibytes servidor leído por segundo durante un período especificado en el `--delay` argumento. - - resultado RPS: ¿Cuántas filas colocadas por el servidor al resultado de una consulta por segundo durante un período `--delay` argumento. - - resultado MiB/s. ¿Cuántos mebibytes colocados por el servidor al resultado de una consulta por segundo durante un período especificado en el `--delay` argumento. - -- Percentiles de tiempo de ejecución de consultas. - -## Modo de comparación {#clickhouse-benchmark-comparison-mode} - -`clickhouse-benchmark` puede comparar el rendimiento de dos servidores ClickHouse en ejecución. - -Para utilizar el modo de comparación, especifique los puntos finales de ambos servidores `--host`, `--port` claves. Las claves coinciden entre sí por posición en la lista de argumentos, la primera `--host` se empareja con la primera `--port` y así sucesivamente. `clickhouse-benchmark` establece conexiones a ambos servidores, luego envía consultas. Cada consulta dirigida a un servidor seleccionado al azar. Los resultados se muestran para cada servidor por separado. - -## Ejemplo {#clickhouse-benchmark-example} - -``` bash -$ echo "SELECT * FROM system.numbers LIMIT 10000000 OFFSET 10000000" | clickhouse-benchmark -i 10 -``` - -``` text -Loaded 1 queries. - -Queries executed: 6. - -localhost:9000, queries 6, QPS: 6.153, RPS: 123398340.957, MiB/s: 941.455, result RPS: 61532982.200, result MiB/s: 469.459. - -0.000% 0.159 sec. -10.000% 0.159 sec. -20.000% 0.159 sec. -30.000% 0.160 sec. -40.000% 0.160 sec. -50.000% 0.162 sec. -60.000% 0.164 sec. -70.000% 0.165 sec. -80.000% 0.166 sec. -90.000% 0.166 sec. -95.000% 0.167 sec. -99.000% 0.167 sec. -99.900% 0.167 sec. -99.990% 0.167 sec. - - - -Queries executed: 10. - -localhost:9000, queries 10, QPS: 6.082, RPS: 121959604.568, MiB/s: 930.478, result RPS: 60815551.642, result MiB/s: 463.986. - -0.000% 0.159 sec. -10.000% 0.159 sec. -20.000% 0.160 sec. -30.000% 0.163 sec. -40.000% 0.164 sec. -50.000% 0.165 sec. -60.000% 0.166 sec. -70.000% 0.166 sec. -80.000% 0.167 sec. -90.000% 0.167 sec. -95.000% 0.170 sec. -99.000% 0.172 sec. -99.900% 0.172 sec. -99.990% 0.172 sec. -``` diff --git a/docs/es/operations/utilities/clickhouse-copier.md b/docs/es/operations/utilities/clickhouse-copier.md deleted file mode 100644 index 5717ffaa737..00000000000 --- a/docs/es/operations/utilities/clickhouse-copier.md +++ /dev/null @@ -1,176 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 59 -toc_title: "M\xE9todo de codificaci\xF3n de datos:" ---- - -# Método de codificación de datos: {#clickhouse-copier} - -Copia datos de las tablas de un clúster en tablas de otro (o del mismo) clúster. - -Puede ejecutar varios `clickhouse-copier` instancias en diferentes servidores para realizar el mismo trabajo. ZooKeeper se utiliza para sincronizar los procesos. - -Después de comenzar, `clickhouse-copier`: - -- Se conecta a ZooKeeper y recibe: - - - Copia de trabajos. - - El estado de los trabajos de copia. - -- Realiza los trabajos. - - Cada proceso en ejecución elige el “closest” el fragmento del clúster de origen y copia los datos en el clúster de destino, reafirmando los datos si es necesario. - -`clickhouse-copier` realiza un seguimiento de los cambios en ZooKeeper y los aplica sobre la marcha. - -Para reducir el tráfico de red, recomendamos ejecutar `clickhouse-copier` en el mismo servidor donde se encuentran los datos de origen. - -## Ejecución de Clickhouse-copiadora {#running-clickhouse-copier} - -La utilidad debe ejecutarse manualmente: - -``` bash -$ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir -``` - -Parámetros: - -- `daemon` — Starts `clickhouse-copier` en modo daemon. -- `config` — The path to the `zookeeper.xml` con los parámetros para la conexión a ZooKeeper. -- `task-path` — The path to the ZooKeeper node. This node is used for syncing `clickhouse-copier` procesos y tareas de almacenamiento. Las tareas se almacenan en `$task-path/description`. -- `task-file` — Optional path to file with task configuration for initial upload to ZooKeeper. -- `task-upload-force` — Force upload `task-file` incluso si el nodo ya existe. -- `base-dir` — The path to logs and auxiliary files. When it starts, `clickhouse-copier` crear `clickhouse-copier_YYYYMMHHSS_` subdirectorios en `$base-dir`. Si se omite este parámetro, los directorios se crean en el directorio donde `clickhouse-copier` se puso en marcha. - -## Formato de Zookeeper.XML {#format-of-zookeeper-xml} - -``` xml - - - trace - 100M - 3 - - - - - 127.0.0.1 - 2181 - - - -``` - -## Configuración de tareas de copia {#configuration-of-copying-tasks} - -``` xml - - - - - - false - - 127.0.0.1 - 9000 - - - ... - - - - ... - - - - - 2 - - - - 1 - - - - - 0 - - - - - 3 - - 1 - - - - - - - - source_cluster - test - hits - - - destination_cluster - test - hits2 - - - - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hits2', '{replica}') - PARTITION BY toMonday(date) - ORDER BY (CounterID, EventDate) - - - - jumpConsistentHash(intHash64(UserID), 2) - - - CounterID != 0 - - - - '2018-02-26' - '2018-03-05' - ... - - - - - - ... - - ... - - -``` - -`clickhouse-copier` seguimiento de los cambios en `/task/path/description` y los aplica sobre la marcha. Por ejemplo, si cambia el valor de `max_workers`, el número de procesos que ejecutan tareas también cambiará. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/utils/clickhouse-copier/) diff --git a/docs/es/operations/utilities/clickhouse-local.md b/docs/es/operations/utilities/clickhouse-local.md deleted file mode 100644 index e122f668f53..00000000000 --- a/docs/es/operations/utilities/clickhouse-local.md +++ /dev/null @@ -1,81 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 60 -toc_title: clickhouse-local ---- - -# clickhouse-local {#clickhouse-local} - -El `clickhouse-local` El programa le permite realizar un procesamiento rápido en archivos locales, sin tener que implementar y configurar el servidor ClickHouse. - -Acepta datos que representan tablas y las consulta usando [Nombre de la red inalámbrica (SSID):](../../sql-reference/index.md). - -`clickhouse-local` utiliza el mismo núcleo que el servidor ClickHouse, por lo que es compatible con la mayoría de las características y el mismo conjunto de formatos y motores de tabla. - -Predeterminada `clickhouse-local` no tiene acceso a los datos en el mismo host, pero admite la carga de la configuración del servidor `--config-file` argumento. - -!!! warning "Advertencia" - No se recomienda cargar la configuración del servidor de producción en `clickhouse-local` Porque los datos pueden dañarse en caso de error humano. - -## Uso {#usage} - -Uso básico: - -``` bash -$ clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" -q "query" -``` - -Argumento: - -- `-S`, `--structure` — table structure for input data. -- `-if`, `--input-format` — input format, `TSV` predeterminada. -- `-f`, `--file` — path to data, `stdin` predeterminada. -- `-q` `--query` — queries to execute with `;` como delimitador. -- `-N`, `--table` — table name where to put output data, `table` predeterminada. -- `-of`, `--format`, `--output-format` — output format, `TSV` predeterminada. -- `--stacktrace` — whether to dump debug output in case of exception. -- `--verbose` — more details on query execution. -- `-s` — disables `stderr` tala. -- `--config-file` — path to configuration file in same format as for ClickHouse server, by default the configuration empty. -- `--help` — arguments references for `clickhouse-local`. - -También hay argumentos para cada variable de configuración de ClickHouse que se usan más comúnmente en lugar de `--config-file`. - -## Ejemplos {#examples} - -``` bash -$ echo -e "1,2\n3,4" | clickhouse-local -S "a Int64, b Int64" -if "CSV" -q "SELECT * FROM table" -Read 2 rows, 32.00 B in 0.000 sec., 5182 rows/sec., 80.97 KiB/sec. -1 2 -3 4 -``` - -El ejemplo anterior es el mismo que: - -``` bash -$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table" -Read 2 rows, 32.00 B in 0.000 sec., 4987 rows/sec., 77.93 KiB/sec. -1 2 -3 4 -``` - -Ahora vamos a usuario de memoria de salida para cada usuario de Unix: - -``` bash -$ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' | clickhouse-local -S "user String, mem Float64" -q "SELECT user, round(sum(mem), 2) as memTotal FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty" -``` - -``` text -Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. -┏━━━━━━━━━━┳━━━━━━━━━━┓ -┃ user ┃ memTotal ┃ -┡━━━━━━━━━━╇━━━━━━━━━━┩ -│ bayonet │ 113.5 │ -├──────────┼──────────┤ -│ root │ 8.8 │ -├──────────┼──────────┤ -... -``` - -[Artículo Original](https://clickhouse.tech/docs/en/operations/utils/clickhouse-local/) diff --git a/docs/es/operations/utilities/index.md b/docs/es/operations/utilities/index.md deleted file mode 100644 index a69397a326c..00000000000 --- a/docs/es/operations/utilities/index.md +++ /dev/null @@ -1,15 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Utilidad -toc_priority: 56 -toc_title: "Descripci\xF3n" ---- - -# Utilidad ClickHouse {#clickhouse-utility} - -- [Sistema abierto.](clickhouse-local.md#clickhouse-local) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` hace esto. -- [Método de codificación de datos:](clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. -- [Sistema abierto.](clickhouse-benchmark.md) — Loads server with the custom queries and settings. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/utils/) diff --git a/docs/es/roadmap.md b/docs/es/roadmap.md deleted file mode 100644 index 60db1c608df..00000000000 --- a/docs/es/roadmap.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -machine_translated: true ---- - -# Hoja De Ruta {#roadmap} - -## Q1 2020 {#q1-2020} - -- Control de acceso basado en roles - -## Q2 2020 {#q2-2020} - -- Integración con servicios de autenticación externos -- Grupos de recursos para una distribución más precisa de la capacidad del clúster entre los usuarios - -{## [Artículo Original](https://clickhouse.tech/docs/es/roadmap/) ##} diff --git a/docs/es/sql-reference/aggregate-functions/combinators.md b/docs/es/sql-reference/aggregate-functions/combinators.md deleted file mode 100644 index c9fdcb9478f..00000000000 --- a/docs/es/sql-reference/aggregate-functions/combinators.md +++ /dev/null @@ -1,245 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: Combinadores ---- - -# Combinadores de funciones agregadas {#aggregate_functions_combinators} - -El nombre de una función agregada puede tener un sufijo anexado. Esto cambia la forma en que funciona la función de agregado. - -## -Si {#agg-functions-combinator-if} - -The suffix -If can be appended to the name of any aggregate function. In this case, the aggregate function accepts an extra argument – a condition (Uint8 type). The aggregate function processes only the rows that trigger the condition. If the condition was not triggered even once, it returns a default value (usually zeros or empty strings). - -Ejemplos: `sumIf(column, cond)`, `countIf(cond)`, `avgIf(x, cond)`, `quantilesTimingIf(level1, level2)(x, cond)`, `argMinIf(arg, val, cond)` y así sucesivamente. - -Con las funciones de agregado condicional, puede calcular agregados para varias condiciones a la vez, sin utilizar subconsultas y `JOIN`Por ejemplo, en Yandex.Metrica, las funciones de agregado condicional se utilizan para implementar la funcionalidad de comparación de segmentos. - -## -Matriz {#agg-functions-combinator-array} - -El sufijo -Array se puede agregar a cualquier función agregada. En este caso, la función de agregado toma argumentos del ‘Array(T)’ tipo (arrays) en lugar de ‘T’ argumentos de tipo. Si la función de agregado acepta varios argumentos, deben ser matrices de igual longitud. Al procesar matrices, la función de agregado funciona como la función de agregado original en todos los elementos de la matriz. - -Ejemplo 1: `sumArray(arr)` - Totales de todos los elementos de todos ‘arr’ matriz. En este ejemplo, podría haber sido escrito más simplemente: `sum(arraySum(arr))`. - -Ejemplo 2: `uniqArray(arr)` – Counts the number of unique elements in all ‘arr’ matriz. Esto podría hacerse de una manera más fácil: `uniq(arrayJoin(arr))`, pero no siempre es posible agregar ‘arrayJoin’ a una consulta. - --If y -Array se pueden combinar. Obstante, ‘Array’ debe venir primero, entonces ‘If’. Ejemplos: `uniqArrayIf(arr, cond)`, `quantilesTimingArrayIf(level1, level2)(arr, cond)`. Debido a este pedido, el ‘cond’ argumento no será una matriz. - -## -Estado {#agg-functions-combinator-state} - -Si aplica este combinador, la función de agregado no devuelve el valor resultante (como el número de valores únicos para el [uniq](reference.md#agg_function-uniq) función), pero un estado intermedio de la agregación (para `uniq`, esta es la tabla hash para calcular el número de valores únicos). Este es un `AggregateFunction(...)` que puede ser utilizado para su posterior procesamiento o almacenado en una tabla para terminar de agregar más tarde. - -Para trabajar con estos estados, use: - -- [AgregaciónMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) motor de mesa. -- [finalizeAggregation](../../sql-reference/functions/other-functions.md#function-finalizeaggregation) función. -- [runningAccumulate](../../sql-reference/functions/other-functions.md#function-runningaccumulate) función. -- [-Fusionar](#aggregate_functions_combinators-merge) combinador. -- [-MergeState](#aggregate_functions_combinators-mergestate) combinador. - -## -Fusionar {#aggregate_functions_combinators-merge} - -Si aplica este combinador, la función de agregado toma el estado de agregación intermedio como argumento, combina los estados para finalizar la agregación y devuelve el valor resultante. - -## -MergeState {#aggregate_functions_combinators-mergestate} - -Combina los estados de agregación intermedios de la misma manera que el combinador -Merge. Sin embargo, no devuelve el valor resultante, sino un estado de agregación intermedio, similar al combinador -State. - -## -ForEach {#agg-functions-combinator-foreach} - -Convierte una función de agregado para tablas en una función de agregado para matrices que agrega los elementos de matriz correspondientes y devuelve una matriz de resultados. Por ejemplo, `sumForEach` para las matrices `[1, 2]`, `[3, 4, 5]`y`[6, 7]`devuelve el resultado `[10, 13, 5]` después de agregar los elementos de la matriz correspondientes. - -## -OPor defecto {#agg-functions-combinator-ordefault} - -Cambia el comportamiento de una función agregada. - -Si una función agregada no tiene valores de entrada, con este combinador devuelve el valor predeterminado para su tipo de datos de retorno. Se aplica a las funciones agregadas que pueden tomar datos de entrada vacíos. - -`-OrDefault` se puede utilizar con otros combinadores. - -**Sintaxis** - -``` sql -OrDefault(x) -``` - -**Parámetros** - -- `x` — Aggregate function parameters. - -**Valores devueltos** - -Devuelve el valor predeterminado del tipo devuelto de una función de agregado si no hay nada que agregar. - -El tipo depende de la función de agregado utilizada. - -**Ejemplo** - -Consulta: - -``` sql -SELECT avg(number), avgOrDefault(number) FROM numbers(0) -``` - -Resultado: - -``` text -┌─avg(number)─┬─avgOrDefault(number)─┐ -│ nan │ 0 │ -└─────────────┴──────────────────────┘ -``` - -También `-OrDefault` se puede utilizar con otros combinadores. Es útil cuando la función de agregado no acepta la entrada vacía. - -Consulta: - -``` sql -SELECT avgOrDefaultIf(x, x > 10) -FROM -( - SELECT toDecimal32(1.23, 2) AS x -) -``` - -Resultado: - -``` text -┌─avgOrDefaultIf(x, greater(x, 10))─┐ -│ 0.00 │ -└───────────────────────────────────┘ -``` - -## -OrNull {#agg-functions-combinator-ornull} - -Cambia el comportamiento de una función agregada. - -Este combinador convierte un resultado de una función agregada en [NULL](../data-types/nullable.md) tipo de datos. Si la función de agregado no tiene valores para calcular devuelve [NULL](../syntax.md#null-literal). - -`-OrNull` se puede utilizar con otros combinadores. - -**Sintaxis** - -``` sql -OrNull(x) -``` - -**Parámetros** - -- `x` — Aggregate function parameters. - -**Valores devueltos** - -- El resultado de la función de agregado, convertida a la `Nullable` tipo de datos. -- `NULL`, si no hay nada que agregar. - -Tipo: `Nullable(aggregate function return type)`. - -**Ejemplo** - -Añadir `-orNull` hasta el final de la función agregada. - -Consulta: - -``` sql -SELECT sumOrNull(number), toTypeName(sumOrNull(number)) FROM numbers(10) WHERE number > 10 -``` - -Resultado: - -``` text -┌─sumOrNull(number)─┬─toTypeName(sumOrNull(number))─┐ -│ ᴺᵁᴸᴸ │ Nullable(UInt64) │ -└───────────────────┴───────────────────────────────┘ -``` - -También `-OrNull` se puede utilizar con otros combinadores. Es útil cuando la función de agregado no acepta la entrada vacía. - -Consulta: - -``` sql -SELECT avgOrNullIf(x, x > 10) -FROM -( - SELECT toDecimal32(1.23, 2) AS x -) -``` - -Resultado: - -``` text -┌─avgOrNullIf(x, greater(x, 10))─┐ -│ ᴺᵁᴸᴸ │ -└────────────────────────────────┘ -``` - -## -Remuestrear {#agg-functions-combinator-resample} - -Permite dividir los datos en grupos y, a continuación, agregar por separado los datos de esos grupos. Los grupos se crean dividiendo los valores de una columna en intervalos. - -``` sql -Resample(start, end, step)(, resampling_key) -``` - -**Parámetros** - -- `start` — Starting value of the whole required interval for `resampling_key` valor. -- `stop` — Ending value of the whole required interval for `resampling_key` valor. Todo el intervalo no incluye el `stop` valor `[start, stop)`. -- `step` — Step for separating the whole interval into subintervals. The `aggFunction` se ejecuta sobre cada uno de esos subintervalos de forma independiente. -- `resampling_key` — Column whose values are used for separating data into intervals. -- `aggFunction_params` — `aggFunction` parámetros. - -**Valores devueltos** - -- Matriz de `aggFunction` resultados para cada subintervalo. - -**Ejemplo** - -Considere el `people` con los siguientes datos: - -``` text -┌─name───┬─age─┬─wage─┐ -│ John │ 16 │ 10 │ -│ Alice │ 30 │ 15 │ -│ Mary │ 35 │ 8 │ -│ Evelyn │ 48 │ 11.5 │ -│ David │ 62 │ 9.9 │ -│ Brian │ 60 │ 16 │ -└────────┴─────┴──────┘ -``` - -Obtengamos los nombres de las personas cuya edad se encuentra en los intervalos de `[30,60)` y `[60,75)`. Como usamos la representación entera para la edad, obtenemos edades en el `[30, 59]` y `[60,74]` intervalo. - -Para agregar nombres en una matriz, usamos el [Método de codificación de datos:](reference.md#agg_function-grouparray) función de agregado. Se necesita un argumento. En nuestro caso, es el `name` columna. El `groupArrayResample` función debe utilizar el `age` columna para agregar nombres por edad. Para definir los intervalos requeridos, pasamos el `30, 75, 30` discusiones sobre el `groupArrayResample` función. - -``` sql -SELECT groupArrayResample(30, 75, 30)(name, age) FROM people -``` - -``` text -┌─groupArrayResample(30, 75, 30)(name, age)─────┐ -│ [['Alice','Mary','Evelyn'],['David','Brian']] │ -└───────────────────────────────────────────────┘ -``` - -Considera los resultados. - -`Jonh` est? fuera de la muestra porque es demasiado joven. Otras personas se distribuyen de acuerdo con los intervalos de edad especificados. - -Ahora vamos a contar el número total de personas y su salario promedio en los intervalos de edad especificados. - -``` sql -SELECT - countResample(30, 75, 30)(name, age) AS amount, - avgResample(30, 75, 30)(wage, age) AS avg_wage -FROM people -``` - -``` text -┌─amount─┬─avg_wage──────────────────┐ -│ [3,2] │ [11.5,12.949999809265137] │ -└────────┴───────────────────────────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/query_language/agg_functions/combinators/) diff --git a/docs/es/sql-reference/aggregate-functions/index.md b/docs/es/sql-reference/aggregate-functions/index.md deleted file mode 100644 index 7c7d58d5f94..00000000000 --- a/docs/es/sql-reference/aggregate-functions/index.md +++ /dev/null @@ -1,62 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Funciones agregadas -toc_priority: 33 -toc_title: "Implantaci\xF3n" ---- - -# Funciones agregadas {#aggregate-functions} - -Las funciones agregadas funcionan en el [normal](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) forma esperada por los expertos en bases de datos. - -ClickHouse también es compatible: - -- [Funciones agregadas paramétricas](parametric-functions.md#aggregate_functions_parametric) que aceptan otros parámetros además de las columnas. -- [Combinadores](combinators.md#aggregate_functions_combinators), que cambian el comportamiento de las funciones agregadas. - -## Procesamiento NULL {#null-processing} - -Durante la agregación, todos `NULL`s se omiten. - -**Ejemplos:** - -Considere esta tabla: - -``` text -┌─x─┬────y─┐ -│ 1 │ 2 │ -│ 2 │ ᴺᵁᴸᴸ │ -│ 3 │ 2 │ -│ 3 │ 3 │ -│ 3 │ ᴺᵁᴸᴸ │ -└───┴──────┘ -``` - -Digamos que necesita sumar los valores en el `y` columna: - -``` sql -SELECT sum(y) FROM t_null_big -``` - - ┌─sum(y)─┐ - │ 7 │ - └────────┘ - -El `sum` función interpreta `NULL` como `0`. En particular, esto significa que si la función recibe la entrada de una selección donde todos los valores son `NULL`, entonces el resultado será `0`, ni `NULL`. - -Ahora puedes usar el `groupArray` función para crear una matriz a partir de la `y` columna: - -``` sql -SELECT groupArray(y) FROM t_null_big -``` - -``` text -┌─groupArray(y)─┐ -│ [2,2,3] │ -└───────────────┘ -``` - -`groupArray` no incluye `NULL` en la matriz resultante. - -[Artículo Original](https://clickhouse.tech/docs/en/query_language/agg_functions/) diff --git a/docs/es/sql-reference/aggregate-functions/parametric-functions.md b/docs/es/sql-reference/aggregate-functions/parametric-functions.md deleted file mode 100644 index ea32920401b..00000000000 --- a/docs/es/sql-reference/aggregate-functions/parametric-functions.md +++ /dev/null @@ -1,499 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: "Param\xE9trico" ---- - -# Funciones agregadas paramétricas {#aggregate_functions_parametric} - -Some aggregate functions can accept not only argument columns (used for compression), but a set of parameters – constants for initialization. The syntax is two pairs of brackets instead of one. The first is for parameters, and the second is for arguments. - -## histograma {#histogram} - -Calcula un histograma adaptativo. No garantiza resultados precisos. - -``` sql -histogram(number_of_bins)(values) -``` - -Las funciones utiliza [Un algoritmo de árbol de decisión paralelo de transmisión](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf). Los bordes de los contenedores de histograma se ajustan a medida que los nuevos datos entran en una función. En caso común, los anchos de los contenedores no son iguales. - -**Parámetros** - -`number_of_bins` — Upper limit for the number of bins in the histogram. The function automatically calculates the number of bins. It tries to reach the specified number of bins, but if it fails, it uses fewer bins. -`values` — [Expresion](../syntax.md#syntax-expressions) resultando en valores de entrada. - -**Valores devueltos** - -- [Matriz](../../sql-reference/data-types/array.md) de [Tuples](../../sql-reference/data-types/tuple.md) del siguiente formato: - - ``` - [(lower_1, upper_1, height_1), ... (lower_N, upper_N, height_N)] - ``` - - - `lower` — Lower bound of the bin. - - `upper` — Upper bound of the bin. - - `height` — Calculated height of the bin. - -**Ejemplo** - -``` sql -SELECT histogram(5)(number + 1) -FROM ( - SELECT * - FROM system.numbers - LIMIT 20 -) -``` - -``` text -┌─histogram(5)(plus(number, 1))───────────────────────────────────────────┐ -│ [(1,4.5,4),(4.5,8.5,4),(8.5,12.75,4.125),(12.75,17,4.625),(17,20,3.25)] │ -└─────────────────────────────────────────────────────────────────────────┘ -``` - -Puede visualizar un histograma con el [Bar](../../sql-reference/functions/other-functions.md#function-bar) función, por ejemplo: - -``` sql -WITH histogram(5)(rand() % 100) AS hist -SELECT - arrayJoin(hist).3 AS height, - bar(height, 0, 6, 5) AS bar -FROM -( - SELECT * - FROM system.numbers - LIMIT 20 -) -``` - -``` text -┌─height─┬─bar───┐ -│ 2.125 │ █▋ │ -│ 3.25 │ ██▌ │ -│ 5.625 │ ████▏ │ -│ 5.625 │ ████▏ │ -│ 3.375 │ ██▌ │ -└────────┴───────┘ -``` - -En este caso, debe recordar que no conoce los bordes del contenedor del histograma. - -## sequenceMatch(pattern)(timestamp, cond1, cond2, …) {#function-sequencematch} - -Comprueba si la secuencia contiene una cadena de eventos que coincida con el patrón. - -``` sql -sequenceMatch(pattern)(timestamp, cond1, cond2, ...) -``` - -!!! warning "Advertencia" - Los eventos que ocurren en el mismo segundo pueden estar en la secuencia en un orden indefinido que afecta el resultado. - -**Parámetros** - -- `pattern` — Pattern string. See [Sintaxis de patrón](#sequence-function-pattern-syntax). - -- `timestamp` — Column considered to contain time data. Typical data types are `Date` y `DateTime`. También puede utilizar cualquiera de los [UInt](../../sql-reference/data-types/int-uint.md) tipos de datos. - -- `cond1`, `cond2` — Conditions that describe the chain of events. Data type: `UInt8`. Puede pasar hasta 32 argumentos de condición. La función sólo tiene en cuenta los eventos descritos en estas condiciones. Si la secuencia contiene datos que no se describen en una condición, la función los omite. - -**Valores devueltos** - -- 1, si el patrón coincide. -- 0, si el patrón no coincide. - -Tipo: `UInt8`. - - -**Sintaxis de patrón** - -- `(?N)` — Matches the condition argument at position `N`. Las condiciones están numeradas en el `[1, 32]` gama. Por ejemplo, `(?1)` coincide con el argumento pasado al `cond1` parámetro. - -- `.*` — Matches any number of events. You don't need conditional arguments to match this element of the pattern. - -- `(?t operator value)` — Sets the time in seconds that should separate two events. For example, pattern `(?1)(?t>1800)(?2)` coincide con los eventos que ocurren a más de 1800 segundos el uno del otro. Un número arbitrario de cualquier evento puede estar entre estos eventos. Puede usar el `>=`, `>`, `<`, `<=` operador. - -**Ejemplos** - -Considere los datos en el `t` tabla: - -``` text -┌─time─┬─number─┐ -│ 1 │ 1 │ -│ 2 │ 3 │ -│ 3 │ 2 │ -└──────┴────────┘ -``` - -Realizar la consulta: - -``` sql -SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2) FROM t -``` - -``` text -┌─sequenceMatch('(?1)(?2)')(time, equals(number, 1), equals(number, 2))─┐ -│ 1 │ -└───────────────────────────────────────────────────────────────────────┘ -``` - -La función encontró la cadena de eventos donde el número 2 sigue al número 1. Se saltó el número 3 entre ellos, porque el número no se describe como un evento. Si queremos tener en cuenta este número al buscar la cadena de eventos dada en el ejemplo, debemos establecer una condición para ello. - -``` sql -SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 3) FROM t -``` - -``` text -┌─sequenceMatch('(?1)(?2)')(time, equals(number, 1), equals(number, 2), equals(number, 3))─┐ -│ 0 │ -└──────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -En este caso, la función no pudo encontrar la cadena de eventos que coincida con el patrón, porque el evento para el número 3 ocurrió entre 1 y 2. Si en el mismo caso comprobamos la condición para el número 4, la secuencia coincidiría con el patrón. - -``` sql -SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM t -``` - -``` text -┌─sequenceMatch('(?1)(?2)')(time, equals(number, 1), equals(number, 2), equals(number, 4))─┐ -│ 1 │ -└──────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -**Ver también** - -- [sequenceCount](#function-sequencecount) - -## sequenceCount(pattern)(time, cond1, cond2, …) {#function-sequencecount} - -Cuenta el número de cadenas de eventos que coinciden con el patrón. La función busca cadenas de eventos que no se superponen. Comienza a buscar la siguiente cadena después de que se haga coincidir la cadena actual. - -!!! warning "Advertencia" - Los eventos que ocurren en el mismo segundo pueden estar en la secuencia en un orden indefinido que afecta el resultado. - -``` sql -sequenceCount(pattern)(timestamp, cond1, cond2, ...) -``` - -**Parámetros** - -- `pattern` — Pattern string. See [Sintaxis de patrón](#sequence-function-pattern-syntax). - -- `timestamp` — Column considered to contain time data. Typical data types are `Date` y `DateTime`. También puede utilizar cualquiera de los [UInt](../../sql-reference/data-types/int-uint.md) tipos de datos. - -- `cond1`, `cond2` — Conditions that describe the chain of events. Data type: `UInt8`. Puede pasar hasta 32 argumentos de condición. La función sólo tiene en cuenta los eventos descritos en estas condiciones. Si la secuencia contiene datos que no se describen en una condición, la función los omite. - -**Valores devueltos** - -- Número de cadenas de eventos no superpuestas que coinciden. - -Tipo: `UInt64`. - -**Ejemplo** - -Considere los datos en el `t` tabla: - -``` text -┌─time─┬─number─┐ -│ 1 │ 1 │ -│ 2 │ 3 │ -│ 3 │ 2 │ -│ 4 │ 1 │ -│ 5 │ 3 │ -│ 6 │ 2 │ -└──────┴────────┘ -``` - -Cuente cuántas veces ocurre el número 2 después del número 1 con cualquier cantidad de otros números entre ellos: - -``` sql -SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t -``` - -``` text -┌─sequenceCount('(?1).*(?2)')(time, equals(number, 1), equals(number, 2))─┐ -│ 2 │ -└─────────────────────────────────────────────────────────────────────────┘ -``` - -**Ver también** - -- [sequenceMatch](#function-sequencematch) - -## ventanaEmbudo {#windowfunnel} - -Busca cadenas de eventos en una ventana de tiempo deslizante y calcula el número máximo de eventos que ocurrieron desde la cadena. - -La función funciona de acuerdo con el algoritmo: - -- La función busca datos que desencadenan la primera condición en la cadena y establece el contador de eventos en 1. Este es el momento en que comienza la ventana deslizante. - -- Si los eventos de la cadena ocurren secuencialmente dentro de la ventana, el contador se incrementa. Si se interrumpe la secuencia de eventos, el contador no se incrementa. - -- Si los datos tienen varias cadenas de eventos en diferentes puntos de finalización, la función solo generará el tamaño de la cadena más larga. - -**Sintaxis** - -``` sql -windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN) -``` - -**Parámetros** - -- `window` — Length of the sliding window in seconds. -- `mode` - Es un argumento opcional. - - `'strict'` - Cuando el `'strict'` se establece, windowFunnel() aplica condiciones solo para los valores únicos. -- `timestamp` — Name of the column containing the timestamp. Data types supported: [Fecha](../../sql-reference/data-types/date.md), [FechaHora](../../sql-reference/data-types/datetime.md#data_type-datetime) y otros tipos de enteros sin signo (tenga en cuenta que aunque timestamp admite el `UInt64` tipo, su valor no puede exceder el máximo de Int64, que es 2 ^ 63 - 1). -- `cond` — Conditions or data describing the chain of events. [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valor devuelto** - -El número máximo de condiciones desencadenadas consecutivas de la cadena dentro de la ventana de tiempo deslizante. -Se analizan todas las cadenas en la selección. - -Tipo: `Integer`. - -**Ejemplo** - -Determine si un período de tiempo establecido es suficiente para que el usuario seleccione un teléfono y lo compre dos veces en la tienda en línea. - -Establezca la siguiente cadena de eventos: - -1. El usuario inició sesión en su cuenta en la tienda (`eventID = 1003`). -2. El usuario busca un teléfono (`eventID = 1007, product = 'phone'`). -3. El usuario realizó un pedido (`eventID = 1009`). -4. El usuario volvió a realizar el pedido (`eventID = 1010`). - -Tabla de entrada: - -``` text -┌─event_date─┬─user_id─┬───────────timestamp─┬─eventID─┬─product─┐ -│ 2019-01-28 │ 1 │ 2019-01-29 10:00:00 │ 1003 │ phone │ -└────────────┴─────────┴─────────────────────┴─────────┴─────────┘ -┌─event_date─┬─user_id─┬───────────timestamp─┬─eventID─┬─product─┐ -│ 2019-01-31 │ 1 │ 2019-01-31 09:00:00 │ 1007 │ phone │ -└────────────┴─────────┴─────────────────────┴─────────┴─────────┘ -┌─event_date─┬─user_id─┬───────────timestamp─┬─eventID─┬─product─┐ -│ 2019-01-30 │ 1 │ 2019-01-30 08:00:00 │ 1009 │ phone │ -└────────────┴─────────┴─────────────────────┴─────────┴─────────┘ -┌─event_date─┬─user_id─┬───────────timestamp─┬─eventID─┬─product─┐ -│ 2019-02-01 │ 1 │ 2019-02-01 08:00:00 │ 1010 │ phone │ -└────────────┴─────────┴─────────────────────┴─────────┴─────────┘ -``` - -Averigüe hasta qué punto el usuario `user_id` podría atravesar la cadena en un período de enero a febrero de 2019. - -Consulta: - -``` sql -SELECT - level, - count() AS c -FROM -( - SELECT - user_id, - windowFunnel(6048000000000000)(timestamp, eventID = 1003, eventID = 1009, eventID = 1007, eventID = 1010) AS level - FROM trend - WHERE (event_date >= '2019-01-01') AND (event_date <= '2019-02-02') - GROUP BY user_id -) -GROUP BY level -ORDER BY level ASC -``` - -Resultado: - -``` text -┌─level─┬─c─┐ -│ 4 │ 1 │ -└───────┴───┘ -``` - -## retención {#retention} - -La función toma como argumentos un conjunto de condiciones de 1 a 32 argumentos de tipo `UInt8` que indican si se cumplió una determinada condición para el evento. -Cualquier condición se puede especificar como un argumento (como en [WHERE](../../sql-reference/statements/select/where.md#select-where)). - -Las condiciones, excepto la primera, se aplican en pares: el resultado del segundo será verdadero si el primero y el segundo son verdaderos, del tercero si el primero y el fird son verdaderos, etc. - -**Sintaxis** - -``` sql -retention(cond1, cond2, ..., cond32); -``` - -**Parámetros** - -- `cond` — an expression that returns a `UInt8` resultado (1 o 0). - -**Valor devuelto** - -La matriz de 1 o 0. - -- 1 — condition was met for the event. -- 0 — condition wasn't met for the event. - -Tipo: `UInt8`. - -**Ejemplo** - -Consideremos un ejemplo de cálculo del `retention` función para determinar el tráfico del sitio. - -**1.** Сreate a table to illustrate an example. - -``` sql -CREATE TABLE retention_test(date Date, uid Int32) ENGINE = Memory; - -INSERT INTO retention_test SELECT '2020-01-01', number FROM numbers(5); -INSERT INTO retention_test SELECT '2020-01-02', number FROM numbers(10); -INSERT INTO retention_test SELECT '2020-01-03', number FROM numbers(15); -``` - -Tabla de entrada: - -Consulta: - -``` sql -SELECT * FROM retention_test -``` - -Resultado: - -``` text -┌───────date─┬─uid─┐ -│ 2020-01-01 │ 0 │ -│ 2020-01-01 │ 1 │ -│ 2020-01-01 │ 2 │ -│ 2020-01-01 │ 3 │ -│ 2020-01-01 │ 4 │ -└────────────┴─────┘ -┌───────date─┬─uid─┐ -│ 2020-01-02 │ 0 │ -│ 2020-01-02 │ 1 │ -│ 2020-01-02 │ 2 │ -│ 2020-01-02 │ 3 │ -│ 2020-01-02 │ 4 │ -│ 2020-01-02 │ 5 │ -│ 2020-01-02 │ 6 │ -│ 2020-01-02 │ 7 │ -│ 2020-01-02 │ 8 │ -│ 2020-01-02 │ 9 │ -└────────────┴─────┘ -┌───────date─┬─uid─┐ -│ 2020-01-03 │ 0 │ -│ 2020-01-03 │ 1 │ -│ 2020-01-03 │ 2 │ -│ 2020-01-03 │ 3 │ -│ 2020-01-03 │ 4 │ -│ 2020-01-03 │ 5 │ -│ 2020-01-03 │ 6 │ -│ 2020-01-03 │ 7 │ -│ 2020-01-03 │ 8 │ -│ 2020-01-03 │ 9 │ -│ 2020-01-03 │ 10 │ -│ 2020-01-03 │ 11 │ -│ 2020-01-03 │ 12 │ -│ 2020-01-03 │ 13 │ -│ 2020-01-03 │ 14 │ -└────────────┴─────┘ -``` - -**2.** Agrupar usuarios por ID único `uid` utilizando el `retention` función. - -Consulta: - -``` sql -SELECT - uid, - retention(date = '2020-01-01', date = '2020-01-02', date = '2020-01-03') AS r -FROM retention_test -WHERE date IN ('2020-01-01', '2020-01-02', '2020-01-03') -GROUP BY uid -ORDER BY uid ASC -``` - -Resultado: - -``` text -┌─uid─┬─r───────┐ -│ 0 │ [1,1,1] │ -│ 1 │ [1,1,1] │ -│ 2 │ [1,1,1] │ -│ 3 │ [1,1,1] │ -│ 4 │ [1,1,1] │ -│ 5 │ [0,0,0] │ -│ 6 │ [0,0,0] │ -│ 7 │ [0,0,0] │ -│ 8 │ [0,0,0] │ -│ 9 │ [0,0,0] │ -│ 10 │ [0,0,0] │ -│ 11 │ [0,0,0] │ -│ 12 │ [0,0,0] │ -│ 13 │ [0,0,0] │ -│ 14 │ [0,0,0] │ -└─────┴─────────┘ -``` - -**3.** Calcule el número total de visitas al sitio por día. - -Consulta: - -``` sql -SELECT - sum(r[1]) AS r1, - sum(r[2]) AS r2, - sum(r[3]) AS r3 -FROM -( - SELECT - uid, - retention(date = '2020-01-01', date = '2020-01-02', date = '2020-01-03') AS r - FROM retention_test - WHERE date IN ('2020-01-01', '2020-01-02', '2020-01-03') - GROUP BY uid -) -``` - -Resultado: - -``` text -┌─r1─┬─r2─┬─r3─┐ -│ 5 │ 5 │ 5 │ -└────┴────┴────┘ -``` - -Donde: - -- `r1`- el número de visitantes únicos que visitaron el sitio durante 2020-01-01 (la `cond1` condición). -- `r2`- el número de visitantes únicos que visitaron el sitio durante un período de tiempo específico entre 2020-01-01 y 2020-01-02 (`cond1` y `cond2` condición). -- `r3`- el número de visitantes únicos que visitaron el sitio durante un período de tiempo específico entre 2020-01-01 y 2020-01-03 (`cond1` y `cond3` condición). - -## UniqUpTo(N)(x) {#uniquptonx} - -Calculates the number of different argument values ​​if it is less than or equal to N. If the number of different argument values is greater than N, it returns N + 1. - -Recomendado para usar con Ns pequeños, hasta 10. El valor máximo de N es 100. - -Para el estado de una función agregada, utiliza la cantidad de memoria igual a 1 + N \* el tamaño de un valor de bytes. -Para las cadenas, almacena un hash no criptográfico de 8 bytes. Es decir, el cálculo se aproxima a las cadenas. - -La función también funciona para varios argumentos. - -Funciona lo más rápido posible, excepto en los casos en que se usa un valor N grande y el número de valores únicos es ligeramente menor que N. - -Ejemplo de uso: - -``` text -Problem: Generate a report that shows only keywords that produced at least 5 unique users. -Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= 5 -``` - -[Artículo Original](https://clickhouse.tech/docs/en/query_language/agg_functions/parametric_functions/) - -## sumMapFiltered(keys_to_keep)(claves, valores) {#summapfilteredkeys-to-keepkeys-values} - -El mismo comportamiento que [sumMap](reference.md#agg_functions-summap) excepto que una matriz de claves se pasa como un parámetro. Esto puede ser especialmente útil cuando se trabaja con una alta cardinalidad de claves. diff --git a/docs/es/sql-reference/aggregate-functions/reference.md b/docs/es/sql-reference/aggregate-functions/reference.md deleted file mode 100644 index 572c4d01051..00000000000 --- a/docs/es/sql-reference/aggregate-functions/reference.md +++ /dev/null @@ -1,1914 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: Referencia ---- - -# Referencia de función agregada {#aggregate-functions-reference} - -## contar {#agg_function-count} - -Cuenta el número de filas o valores no NULL. - -ClickHouse admite las siguientes sintaxis para `count`: -- `count(expr)` o `COUNT(DISTINCT expr)`. -- `count()` o `COUNT(*)`. El `count()` la sintaxis es específica de ClickHouse. - -**Parámetros** - -La función puede tomar: - -- Cero parámetros. -- Una [expresion](../syntax.md#syntax-expressions). - -**Valor devuelto** - -- Si se llama a la función sin parámetros, cuenta el número de filas. -- Si el [expresion](../syntax.md#syntax-expressions) se pasa, entonces la función cuenta cuántas veces esta expresión devuelve no nula. Si la expresión devuelve un [NULL](../../sql-reference/data-types/nullable.md)-type valor, entonces el resultado de `count` no se queda `Nullable`. La función devuelve 0 si la expresión devuelta `NULL` para todas las filas. - -En ambos casos el tipo del valor devuelto es [UInt64](../../sql-reference/data-types/int-uint.md). - -**Detalles** - -ClickHouse soporta el `COUNT(DISTINCT ...)` sintaxis. El comportamiento de esta construcción depende del [count_distinct_implementation](../../operations/settings/settings.md#settings-count_distinct_implementation) configuración. Define cuál de las [uniq\*](#agg_function-uniq) se utiliza para realizar la operación. El valor predeterminado es el [uniqExact](#agg_function-uniqexact) función. - -El `SELECT count() FROM table` consulta no está optimizado, porque el número de entradas en la tabla no se almacena por separado. Elige una pequeña columna de la tabla y cuenta el número de valores en ella. - -**Ejemplos** - -Ejemplo 1: - -``` sql -SELECT count() FROM t -``` - -``` text -┌─count()─┐ -│ 5 │ -└─────────┘ -``` - -Ejemplo 2: - -``` sql -SELECT name, value FROM system.settings WHERE name = 'count_distinct_implementation' -``` - -``` text -┌─name──────────────────────────┬─value─────┐ -│ count_distinct_implementation │ uniqExact │ -└───────────────────────────────┴───────────┘ -``` - -``` sql -SELECT count(DISTINCT num) FROM t -``` - -``` text -┌─uniqExact(num)─┐ -│ 3 │ -└────────────────┘ -``` - -Este ejemplo muestra que `count(DISTINCT num)` se realiza por el `uniqExact` función según el `count_distinct_implementation` valor de ajuste. - -## cualquiera (x) {#agg_function-any} - -Selecciona el primer valor encontrado. -La consulta se puede ejecutar en cualquier orden e incluso en un orden diferente cada vez, por lo que el resultado de esta función es indeterminado. -Para obtener un resultado determinado, puede usar el ‘min’ o ‘max’ función en lugar de ‘any’. - -En algunos casos, puede confiar en el orden de ejecución. Esto se aplica a los casos en que SELECT proviene de una subconsulta que usa ORDER BY. - -Cuando un `SELECT` consulta tiene el `GROUP BY` cláusula o al menos una función agregada, ClickHouse (en contraste con MySQL) requiere que todas las expresiones `SELECT`, `HAVING`, y `ORDER BY` las cláusulas pueden calcularse a partir de claves o de funciones agregadas. En otras palabras, cada columna seleccionada de la tabla debe usarse en claves o dentro de funciones agregadas. Para obtener un comportamiento como en MySQL, puede colocar las otras columnas en el `any` función de agregado. - -## Cualquier pesado (x) {#anyheavyx} - -Selecciona un valor que ocurre con frecuencia [pesos pesados](http://www.cs.umd.edu/~samir/498/karp.pdf) algoritmo. Si hay un valor que se produce más de la mitad de los casos en cada uno de los subprocesos de ejecución de la consulta, se devuelve este valor. Normalmente, el resultado es no determinista. - -``` sql -anyHeavy(column) -``` - -**Argumento** - -- `column` – The column name. - -**Ejemplo** - -Tome el [A tiempo](../../getting-started/example-datasets/ontime.md) conjunto de datos y seleccione cualquier valor que ocurra con frecuencia `AirlineID` columna. - -``` sql -SELECT anyHeavy(AirlineID) AS res -FROM ontime -``` - -``` text -┌───res─┐ -│ 19690 │ -└───────┘ -``` - -## Cualquier último (x) {#anylastx} - -Selecciona el último valor encontrado. -El resultado es tan indeterminado como para el `any` función. - -## Método de codificación de datos: {#groupbitand} - -Se aplica bit a bit `AND` para la serie de números. - -``` sql -groupBitAnd(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `UInt*` tipo. - -**Valor de retorno** - -Valor de la `UInt*` tipo. - -**Ejemplo** - -Datos de prueba: - -``` text -binary decimal -00101100 = 44 -00011100 = 28 -00001101 = 13 -01010101 = 85 -``` - -Consulta: - -``` sql -SELECT groupBitAnd(num) FROM t -``` - -Donde `num` es la columna con los datos de prueba. - -Resultado: - -``` text -binary decimal -00000100 = 4 -``` - -## GrupoBitO {#groupbitor} - -Se aplica bit a bit `OR` para la serie de números. - -``` sql -groupBitOr(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `UInt*` tipo. - -**Valor de retorno** - -Valor de la `UInt*` tipo. - -**Ejemplo** - -Datos de prueba: - -``` text -binary decimal -00101100 = 44 -00011100 = 28 -00001101 = 13 -01010101 = 85 -``` - -Consulta: - -``` sql -SELECT groupBitOr(num) FROM t -``` - -Donde `num` es la columna con los datos de prueba. - -Resultado: - -``` text -binary decimal -01111101 = 125 -``` - -## GrupoBitXor {#groupbitxor} - -Se aplica bit a bit `XOR` para la serie de números. - -``` sql -groupBitXor(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `UInt*` tipo. - -**Valor de retorno** - -Valor de la `UInt*` tipo. - -**Ejemplo** - -Datos de prueba: - -``` text -binary decimal -00101100 = 44 -00011100 = 28 -00001101 = 13 -01010101 = 85 -``` - -Consulta: - -``` sql -SELECT groupBitXor(num) FROM t -``` - -Donde `num` es la columna con los datos de prueba. - -Resultado: - -``` text -binary decimal -01101000 = 104 -``` - -## Método de codificación de datos: {#groupbitmap} - -Mapa de bits o cálculos agregados de una columna entera sin signo, devuelve cardinalidad de tipo UInt64, si agrega el sufijo -State, luego devuelve [objeto de mapa de bits](../../sql-reference/functions/bitmap-functions.md). - -``` sql -groupBitmap(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `UInt*` tipo. - -**Valor de retorno** - -Valor de la `UInt64` tipo. - -**Ejemplo** - -Datos de prueba: - -``` text -UserID -1 -1 -2 -3 -``` - -Consulta: - -``` sql -SELECT groupBitmap(UserID) as num FROM t -``` - -Resultado: - -``` text -num -3 -``` - -## Mínimo (x) {#agg_function-min} - -Calcula el mínimo. - -## máximo (x) {#agg_function-max} - -Calcula el máximo. - -## ¿Cómo puedo hacerlo?) {#agg-function-argmin} - -Calcula el ‘arg’ para un valor mínimo ‘val’ valor. Si hay varios valores diferentes de ‘arg’ para valores mínimos de ‘val’, el primero de estos valores encontrados es la salida. - -**Ejemplo:** - -``` text -┌─user─────┬─salary─┐ -│ director │ 5000 │ -│ manager │ 3000 │ -│ worker │ 1000 │ -└──────────┴────────┘ -``` - -``` sql -SELECT argMin(user, salary) FROM salary -``` - -``` text -┌─argMin(user, salary)─┐ -│ worker │ -└──────────────────────┘ -``` - -## Descripción) {#agg-function-argmax} - -Calcula el ‘arg’ para un valor máximo ‘val’ valor. Si hay varios valores diferentes de ‘arg’ para valores máximos de ‘val’, el primero de estos valores encontrados es la salida. - -## suma (x) {#agg_function-sum} - -Calcula la suma. -Solo funciona para números. - -## ¿Cómo puedo obtener más información?) {#sumwithoverflowx} - -Calcula la suma de los números, utilizando el mismo tipo de datos para el resultado que para los parámetros de entrada. Si la suma supera el valor máximo para este tipo de datos, la función devuelve un error. - -Solo funciona para números. - -## Por ejemplo, el valor es el siguiente:)) {#agg_functions-summap} - -Totals el ‘value’ matriz de acuerdo con las claves especificadas en el ‘key’ matriz. -Pasar una tupla de matrices de claves y valores es sinónimo de pasar dos matrices de claves y valores. -El número de elementos en ‘key’ y ‘value’ debe ser el mismo para cada fila que se sume. -Returns a tuple of two arrays: keys in sorted order, and values ​​summed for the corresponding keys. - -Ejemplo: - -``` sql -CREATE TABLE sum_map( - date Date, - timeslot DateTime, - statusMap Nested( - status UInt16, - requests UInt64 - ), - statusMapTuple Tuple(Array(Int32), Array(Int32)) -) ENGINE = Log; -INSERT INTO sum_map VALUES - ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10], ([1, 2, 3], [10, 10, 10])), - ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10], ([3, 4, 5], [10, 10, 10])), - ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10], ([4, 5, 6], [10, 10, 10])), - ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10], ([6, 7, 8], [10, 10, 10])); - -SELECT - timeslot, - sumMap(statusMap.status, statusMap.requests), - sumMap(statusMapTuple) -FROM sum_map -GROUP BY timeslot -``` - -``` text -┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┬─sumMap(statusMapTuple)─────────┐ -│ 2000-01-01 00:00:00 │ ([1,2,3,4,5],[10,10,20,10,10]) │ ([1,2,3,4,5],[10,10,20,10,10]) │ -│ 2000-01-01 00:01:00 │ ([4,5,6,7,8],[10,10,20,10,10]) │ ([4,5,6,7,8],[10,10,20,10,10]) │ -└─────────────────────┴──────────────────────────────────────────────┴────────────────────────────────┘ -``` - -## SkewPop {#skewpop} - -Calcula el [la asimetría](https://en.wikipedia.org/wiki/Skewness) de una secuencia. - -``` sql -skewPop(expr) -``` - -**Parámetros** - -`expr` — [Expresion](../syntax.md#syntax-expressions) devolviendo un número. - -**Valor devuelto** - -The skewness of the given distribution. Type — [Float64](../../sql-reference/data-types/float.md) - -**Ejemplo** - -``` sql -SELECT skewPop(value) FROM series_with_value_column -``` - -## Sistema abierto {#skewsamp} - -Calcula el [asimetría de la muestra](https://en.wikipedia.org/wiki/Skewness) de una secuencia. - -Representa una estimación imparcial de la asimetría de una variable aleatoria si los valores pasados forman su muestra. - -``` sql -skewSamp(expr) -``` - -**Parámetros** - -`expr` — [Expresion](../syntax.md#syntax-expressions) devolviendo un número. - -**Valor devuelto** - -The skewness of the given distribution. Type — [Float64](../../sql-reference/data-types/float.md). Si `n <= 1` (`n` es el tamaño de la muestra), luego la función devuelve `nan`. - -**Ejemplo** - -``` sql -SELECT skewSamp(value) FROM series_with_value_column -``` - -## KurtPop {#kurtpop} - -Calcula el [curtosis](https://en.wikipedia.org/wiki/Kurtosis) de una secuencia. - -``` sql -kurtPop(expr) -``` - -**Parámetros** - -`expr` — [Expresion](../syntax.md#syntax-expressions) devolviendo un número. - -**Valor devuelto** - -The kurtosis of the given distribution. Type — [Float64](../../sql-reference/data-types/float.md) - -**Ejemplo** - -``` sql -SELECT kurtPop(value) FROM series_with_value_column -``` - -## KurtSamp {#kurtsamp} - -Calcula el [curtosis muestra](https://en.wikipedia.org/wiki/Kurtosis) de una secuencia. - -Representa una estimación imparcial de la curtosis de una variable aleatoria si los valores pasados forman su muestra. - -``` sql -kurtSamp(expr) -``` - -**Parámetros** - -`expr` — [Expresion](../syntax.md#syntax-expressions) devolviendo un número. - -**Valor devuelto** - -The kurtosis of the given distribution. Type — [Float64](../../sql-reference/data-types/float.md). Si `n <= 1` (`n` es un tamaño de la muestra), luego la función devuelve `nan`. - -**Ejemplo** - -``` sql -SELECT kurtSamp(value) FROM series_with_value_column -``` - -## Acerca de) {#agg_function-avg} - -Calcula el promedio. -Solo funciona para números. -El resultado es siempre Float64. - -## avgPonderado {#avgweighted} - -Calcula el [media aritmética ponderada](https://en.wikipedia.org/wiki/Weighted_arithmetic_mean). - -**Sintaxis** - -``` sql -avgWeighted(x, weight) -``` - -**Parámetros** - -- `x` — Values. [Entero](../data-types/int-uint.md) o [punto flotante](../data-types/float.md). -- `weight` — Weights of the values. [Entero](../data-types/int-uint.md) o [punto flotante](../data-types/float.md). - -Tipo de `x` y `weight` debe ser el mismo. - -**Valor devuelto** - -- Media ponderada. -- `NaN`. Si todos los pesos son iguales a 0. - -Tipo: [Float64](../data-types/float.md). - -**Ejemplo** - -Consulta: - -``` sql -SELECT avgWeighted(x, w) -FROM values('x Int8, w Int8', (4, 1), (1, 0), (10, 2)) -``` - -Resultado: - -``` text -┌─avgWeighted(x, weight)─┐ -│ 8 │ -└────────────────────────┘ -``` - -## uniq {#agg_function-uniq} - -Calcula el número aproximado de diferentes valores del argumento. - -``` sql -uniq(x[, ...]) -``` - -**Parámetros** - -La función toma un número variable de parámetros. Los parámetros pueden ser `Tuple`, `Array`, `Date`, `DateTime`, `String`, o tipos numéricos. - -**Valor devuelto** - -- A [UInt64](../../sql-reference/data-types/int-uint.md)-tipo número. - -**Detalles de implementación** - -Función: - -- Calcula un hash para todos los parámetros en el agregado, luego lo usa en los cálculos. - -- Utiliza un algoritmo de muestreo adaptativo. Para el estado de cálculo, la función utiliza una muestra de valores hash de elemento de hasta 65536. - - This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions. - -- Proporciona el resultado de forma determinista (no depende del orden de procesamiento de la consulta). - -Recomendamos usar esta función en casi todos los escenarios. - -**Ver también** - -- [uniqCombined](#agg_function-uniqcombined) -- [UniqCombined64](#agg_function-uniqcombined64) -- [uniqHLL12](#agg_function-uniqhll12) -- [uniqExact](#agg_function-uniqexact) - -## uniqCombined {#agg_function-uniqcombined} - -Calcula el número aproximado de diferentes valores de argumento. - -``` sql -uniqCombined(HLL_precision)(x[, ...]) -``` - -El `uniqCombined` es una buena opción para calcular el número de valores diferentes. - -**Parámetros** - -La función toma un número variable de parámetros. Los parámetros pueden ser `Tuple`, `Array`, `Date`, `DateTime`, `String`, o tipos numéricos. - -`HLL_precision` es el logaritmo base-2 del número de células en [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Opcional, puede utilizar la función como `uniqCombined(x[, ...])`. El valor predeterminado para `HLL_precision` es 17, que es efectivamente 96 KiB de espacio (2 ^ 17 celdas, 6 bits cada una). - -**Valor devuelto** - -- Numero [UInt64](../../sql-reference/data-types/int-uint.md)-tipo número. - -**Detalles de implementación** - -Función: - -- Calcula un hash (hash de 64 bits para `String` y 32 bits de lo contrario) para todos los parámetros en el agregado, luego lo usa en los cálculos. - -- Utiliza una combinación de tres algoritmos: matriz, tabla hash e HyperLogLog con una tabla de corrección de errores. - - For a small number of distinct elements, an array is used. When the set size is larger, a hash table is used. For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. - -- Proporciona el resultado de forma determinista (no depende del orden de procesamiento de la consulta). - -!!! note "Nota" - Dado que usa hash de 32 bits para no-`String` tipo, el resultado tendrá un error muy alto para cardinalidades significativamente mayores que `UINT_MAX` (el error aumentará rápidamente después de unas pocas decenas de miles de millones de valores distintos), por lo tanto, en este caso debe usar [UniqCombined64](#agg_function-uniqcombined64) - -En comparación con el [uniq](#agg_function-uniq) función, el `uniqCombined`: - -- Consume varias veces menos memoria. -- Calcula con una precisión varias veces mayor. -- Por lo general, tiene un rendimiento ligeramente menor. En algunos escenarios, `uniqCombined` puede funcionar mejor que `uniq`, por ejemplo, con consultas distribuidas que transmiten un gran número de estados de agregación a través de la red. - -**Ver también** - -- [uniq](#agg_function-uniq) -- [UniqCombined64](#agg_function-uniqcombined64) -- [uniqHLL12](#agg_function-uniqhll12) -- [uniqExact](#agg_function-uniqexact) - -## UniqCombined64 {#agg_function-uniqcombined64} - -Lo mismo que [uniqCombined](#agg_function-uniqcombined), pero utiliza hash de 64 bits para todos los tipos de datos. - -## uniqHLL12 {#agg_function-uniqhll12} - -Calcula el número aproximado de diferentes valores de argumento [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algoritmo. - -``` sql -uniqHLL12(x[, ...]) -``` - -**Parámetros** - -La función toma un número variable de parámetros. Los parámetros pueden ser `Tuple`, `Array`, `Date`, `DateTime`, `String`, o tipos numéricos. - -**Valor devuelto** - -- A [UInt64](../../sql-reference/data-types/int-uint.md)-tipo número. - -**Detalles de implementación** - -Función: - -- Calcula un hash para todos los parámetros en el agregado, luego lo usa en los cálculos. - -- Utiliza el algoritmo HyperLogLog para aproximar el número de valores de argumento diferentes. - - 212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. The result is not very accurate (up to ~10% error) for small data sets (<10K elements). However, the result is fairly accurate for high-cardinality data sets (10K-100M), with a maximum error of ~1.6%. Starting from 100M, the estimation error increases, and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements). - -- Proporciona el resultado determinado (no depende del orden de procesamiento de la consulta). - -No recomendamos usar esta función. En la mayoría de los casos, use el [uniq](#agg_function-uniq) o [uniqCombined](#agg_function-uniqcombined) función. - -**Ver también** - -- [uniq](#agg_function-uniq) -- [uniqCombined](#agg_function-uniqcombined) -- [uniqExact](#agg_function-uniqexact) - -## uniqExact {#agg_function-uniqexact} - -Calcula el número exacto de diferentes valores de argumento. - -``` sql -uniqExact(x[, ...]) -``` - -Utilice el `uniqExact` función si necesita absolutamente un resultado exacto. De lo contrario, use el [uniq](#agg_function-uniq) función. - -El `uniqExact` función utiliza más memoria que `uniq`, porque el tamaño del estado tiene un crecimiento ilimitado a medida que aumenta el número de valores diferentes. - -**Parámetros** - -La función toma un número variable de parámetros. Los parámetros pueden ser `Tuple`, `Array`, `Date`, `DateTime`, `String`, o tipos numéricos. - -**Ver también** - -- [uniq](#agg_function-uniq) -- [uniqCombined](#agg_function-uniqcombined) -- [uniqHLL12](#agg_function-uniqhll12) - -## ¿Cómo puedo hacerlo?) {#agg_function-grouparray} - -Crea una matriz de valores de argumento. -Los valores se pueden agregar a la matriz en cualquier orden (indeterminado). - -La segunda versión (con el `max_size` parámetro) limita el tamaño de la matriz resultante a `max_size` elemento. -Por ejemplo, `groupArray (1) (x)` es equivalente a `[any (x)]`. - -En algunos casos, aún puede confiar en el orden de ejecución. Esto se aplica a los casos en que `SELECT` procede de una subconsulta que utiliza `ORDER BY`. - -## GrupoArrayInsertAt {#grouparrayinsertat} - -Inserta un valor en la matriz en la posición especificada. - -**Sintaxis** - -``` sql -groupArrayInsertAt(default_x, size)(x, pos); -``` - -Si en una consulta se insertan varios valores en la misma posición, la función se comporta de las siguientes maneras: - -- Si se ejecuta una consulta en un solo subproceso, se utiliza el primero de los valores insertados. -- Si una consulta se ejecuta en varios subprocesos, el valor resultante es uno indeterminado de los valores insertados. - -**Parámetros** - -- `x` — Value to be inserted. [Expresion](../syntax.md#syntax-expressions) lo que resulta en uno de los [tipos de datos compatibles](../../sql-reference/data-types/index.md). -- `pos` — Position at which the specified element `x` se va a insertar. La numeración de índices en la matriz comienza desde cero. [UInt32](../../sql-reference/data-types/int-uint.md#uint-ranges). -- `default_x`— Default value for substituting in empty positions. Optional parameter. [Expresion](../syntax.md#syntax-expressions) dando como resultado el tipo de datos configurado para `x` parámetro. Si `default_x` no está definido, el [valores predeterminados](../../sql-reference/statements/create.md#create-default-values) se utilizan. -- `size`— Length of the resulting array. Optional parameter. When using this parameter, the default value `default_x` debe ser especificado. [UInt32](../../sql-reference/data-types/int-uint.md#uint-ranges). - -**Valor devuelto** - -- Matriz con valores insertados. - -Tipo: [Matriz](../../sql-reference/data-types/array.md#data-type-array). - -**Ejemplo** - -Consulta: - -``` sql -SELECT groupArrayInsertAt(toString(number), number * 2) FROM numbers(5); -``` - -Resultado: - -``` text -┌─groupArrayInsertAt(toString(number), multiply(number, 2))─┐ -│ ['0','','1','','2','','3','','4'] │ -└───────────────────────────────────────────────────────────┘ -``` - -Consulta: - -``` sql -SELECT groupArrayInsertAt('-')(toString(number), number * 2) FROM numbers(5); -``` - -Resultado: - -``` text -┌─groupArrayInsertAt('-')(toString(number), multiply(number, 2))─┐ -│ ['0','-','1','-','2','-','3','-','4'] │ -└────────────────────────────────────────────────────────────────┘ -``` - -Consulta: - -``` sql -SELECT groupArrayInsertAt('-', 5)(toString(number), number * 2) FROM numbers(5); -``` - -Resultado: - -``` text -┌─groupArrayInsertAt('-', 5)(toString(number), multiply(number, 2))─┐ -│ ['0','-','1','-','2'] │ -└───────────────────────────────────────────────────────────────────┘ -``` - -Inserción multihilo de elementos en una posición. - -Consulta: - -``` sql -SELECT groupArrayInsertAt(number, 0) FROM numbers_mt(10) SETTINGS max_block_size = 1; -``` - -Como resultado de esta consulta, obtiene un entero aleatorio en el `[0,9]` gama. Por ejemplo: - -``` text -┌─groupArrayInsertAt(number, 0)─┐ -│ [7] │ -└───────────────────────────────┘ -``` - -## groupArrayMovingSum {#agg_function-grouparraymovingsum} - -Calcula la suma móvil de los valores de entrada. - -``` sql -groupArrayMovingSum(numbers_for_summing) -groupArrayMovingSum(window_size)(numbers_for_summing) -``` - -La función puede tomar el tamaño de la ventana como un parámetro. Si no se especifica, la función toma el tamaño de ventana igual al número de filas de la columna. - -**Parámetros** - -- `numbers_for_summing` — [Expresion](../syntax.md#syntax-expressions) dando como resultado un valor de tipo de datos numérico. -- `window_size` — Size of the calculation window. - -**Valores devueltos** - -- Matriz del mismo tamaño y tipo que los datos de entrada. - -**Ejemplo** - -La tabla de ejemplo: - -``` sql -CREATE TABLE t -( - `int` UInt8, - `float` Float32, - `dec` Decimal32(2) -) -ENGINE = TinyLog -``` - -``` text -┌─int─┬─float─┬──dec─┐ -│ 1 │ 1.1 │ 1.10 │ -│ 2 │ 2.2 │ 2.20 │ -│ 4 │ 4.4 │ 4.40 │ -│ 7 │ 7.77 │ 7.77 │ -└─────┴───────┴──────┘ -``` - -Consulta: - -``` sql -SELECT - groupArrayMovingSum(int) AS I, - groupArrayMovingSum(float) AS F, - groupArrayMovingSum(dec) AS D -FROM t -``` - -``` text -┌─I──────────┬─F───────────────────────────────┬─D──────────────────────┐ -│ [1,3,7,14] │ [1.1,3.3000002,7.7000003,15.47] │ [1.10,3.30,7.70,15.47] │ -└────────────┴─────────────────────────────────┴────────────────────────┘ -``` - -``` sql -SELECT - groupArrayMovingSum(2)(int) AS I, - groupArrayMovingSum(2)(float) AS F, - groupArrayMovingSum(2)(dec) AS D -FROM t -``` - -``` text -┌─I──────────┬─F───────────────────────────────┬─D──────────────────────┐ -│ [1,3,6,11] │ [1.1,3.3000002,6.6000004,12.17] │ [1.10,3.30,6.60,12.17] │ -└────────────┴─────────────────────────────────┴────────────────────────┘ -``` - -## Método de codificación de datos: {#agg_function-grouparraymovingavg} - -Calcula la media móvil de los valores de entrada. - -``` sql -groupArrayMovingAvg(numbers_for_summing) -groupArrayMovingAvg(window_size)(numbers_for_summing) -``` - -La función puede tomar el tamaño de la ventana como un parámetro. Si no se especifica, la función toma el tamaño de ventana igual al número de filas de la columna. - -**Parámetros** - -- `numbers_for_summing` — [Expresion](../syntax.md#syntax-expressions) dando como resultado un valor de tipo de datos numérico. -- `window_size` — Size of the calculation window. - -**Valores devueltos** - -- Matriz del mismo tamaño y tipo que los datos de entrada. - -La función utiliza [redondeando hacia cero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero). Trunca los decimales insignificantes para el tipo de datos resultante. - -**Ejemplo** - -La tabla de ejemplo `b`: - -``` sql -CREATE TABLE t -( - `int` UInt8, - `float` Float32, - `dec` Decimal32(2) -) -ENGINE = TinyLog -``` - -``` text -┌─int─┬─float─┬──dec─┐ -│ 1 │ 1.1 │ 1.10 │ -│ 2 │ 2.2 │ 2.20 │ -│ 4 │ 4.4 │ 4.40 │ -│ 7 │ 7.77 │ 7.77 │ -└─────┴───────┴──────┘ -``` - -Consulta: - -``` sql -SELECT - groupArrayMovingAvg(int) AS I, - groupArrayMovingAvg(float) AS F, - groupArrayMovingAvg(dec) AS D -FROM t -``` - -``` text -┌─I─────────┬─F───────────────────────────────────┬─D─────────────────────┐ -│ [0,0,1,3] │ [0.275,0.82500005,1.9250001,3.8675] │ [0.27,0.82,1.92,3.86] │ -└───────────┴─────────────────────────────────────┴───────────────────────┘ -``` - -``` sql -SELECT - groupArrayMovingAvg(2)(int) AS I, - groupArrayMovingAvg(2)(float) AS F, - groupArrayMovingAvg(2)(dec) AS D -FROM t -``` - -``` text -┌─I─────────┬─F────────────────────────────────┬─D─────────────────────┐ -│ [0,1,3,5] │ [0.55,1.6500001,3.3000002,6.085] │ [0.55,1.65,3.30,6.08] │ -└───────────┴──────────────────────────────────┴───────────────────────┘ -``` - -## ¿Cómo puedo obtener más información?) {#groupuniqarrayx-groupuniqarraymax-sizex} - -Crea una matriz a partir de diferentes valores de argumento. El consumo de memoria es el mismo que para el `uniqExact` función. - -La segunda versión (con el `max_size` parámetro) limita el tamaño de la matriz resultante a `max_size` elemento. -Por ejemplo, `groupUniqArray(1)(x)` es equivalente a `[any(x)]`. - -## cuantil {#quantile} - -Calcula un aproximado [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos. - -Esta función se aplica [muestreo de embalses](https://en.wikipedia.org/wiki/Reservoir_sampling) con un tamaño de depósito de hasta 8192 y un generador de números aleatorios para el muestreo. El resultado es no determinista. Para obtener un cuantil exacto, use el [quantileExact](#quantileexact) función. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantile(level)(expr) -``` - -Apodo: `median`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). - -**Valor devuelto** - -- Cuantil aproximado del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─val─┐ -│ 1 │ -│ 1 │ -│ 2 │ -│ 3 │ -└─────┘ -``` - -Consulta: - -``` sql -SELECT quantile(val) FROM t -``` - -Resultado: - -``` text -┌─quantile(val)─┐ -│ 1.5 │ -└───────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileDeterminista {#quantiledeterministic} - -Calcula un aproximado [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos. - -Esta función se aplica [muestreo de embalses](https://en.wikipedia.org/wiki/Reservoir_sampling) con un tamaño de depósito de hasta 8192 y algoritmo determinista de muestreo. El resultado es determinista. Para obtener un cuantil exacto, use el [quantileExact](#quantileexact) función. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileDeterministic(level)(expr, determinator) -``` - -Apodo: `medianDeterministic`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). -- `determinator` — Number whose hash is used instead of a random number generator in the reservoir sampling algorithm to make the result of sampling deterministic. As a determinator you can use any deterministic positive number, for example, a user id or an event id. If the same determinator value occures too often, the function works incorrectly. - -**Valor devuelto** - -- Cuantil aproximado del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─val─┐ -│ 1 │ -│ 1 │ -│ 2 │ -│ 3 │ -└─────┘ -``` - -Consulta: - -``` sql -SELECT quantileDeterministic(val, 1) FROM t -``` - -Resultado: - -``` text -┌─quantileDeterministic(val, 1)─┐ -│ 1.5 │ -└───────────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileExact {#quantileexact} - -Calcula exactamente el [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos. - -To get exact value, all the passed values ​​are combined into an array, which is then partially sorted. Therefore, the function consumes `O(n)` memoria, donde `n` es un número de valores que se pasaron. Sin embargo, para un pequeño número de valores, la función es muy efectiva. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileExact(level)(expr) -``` - -Apodo: `medianExact`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). - -**Valor devuelto** - -- Cuantil del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Consulta: - -``` sql -SELECT quantileExact(number) FROM numbers(10) -``` - -Resultado: - -``` text -┌─quantileExact(number)─┐ -│ 5 │ -└───────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileExactWeighted {#quantileexactweighted} - -Calcula exactamente el [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos, teniendo en cuenta el peso de cada elemento. - -To get exact value, all the passed values ​​are combined into an array, which is then partially sorted. Each value is counted with its weight, as if it is present `weight` times. A hash table is used in the algorithm. Because of this, if the passed values ​​are frequently repeated, the function consumes less RAM than [quantileExact](#quantileexact). Puede usar esta función en lugar de `quantileExact` y especifique el peso 1. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileExactWeighted(level)(expr, weight) -``` - -Apodo: `medianExactWeighted`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). -- `weight` — Column with weights of sequence members. Weight is a number of value occurrences. - -**Valor devuelto** - -- Cuantil del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─n─┬─val─┐ -│ 0 │ 3 │ -│ 1 │ 2 │ -│ 2 │ 1 │ -│ 5 │ 4 │ -└───┴─────┘ -``` - -Consulta: - -``` sql -SELECT quantileExactWeighted(n, val) FROM t -``` - -Resultado: - -``` text -┌─quantileExactWeighted(n, val)─┐ -│ 1 │ -└───────────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileTiming {#quantiletiming} - -Con la precisión determinada calcula el [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos. - -El resultado es determinista (no depende del orden de procesamiento de la consulta). La función está optimizada para trabajar con secuencias que describen distribuciones como tiempos de carga de páginas web o tiempos de respuesta de back-end. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileTiming(level)(expr) -``` - -Apodo: `medianTiming`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). - -- `expr` — [Expresion](../syntax.md#syntax-expressions) sobre una columna valores que devuelven un [Flotante\*](../../sql-reference/data-types/float.md)-tipo número. - - - If negative values are passed to the function, the behavior is undefined. - - If the value is greater than 30,000 (a page loading time of more than 30 seconds), it is assumed to be 30,000. - -**Exactitud** - -El cálculo es preciso si: - -- El número total de valores no supera los 5670. -- El número total de valores supera los 5670, pero el tiempo de carga de la página es inferior a 1024 ms. - -De lo contrario, el resultado del cálculo se redondea al múltiplo más cercano de 16 ms. - -!!! note "Nota" - Para calcular los cuantiles de tiempo de carga de la página, esta función es más efectiva y precisa que [cuantil](#quantile). - -**Valor devuelto** - -- Cuantil del nivel especificado. - -Tipo: `Float32`. - -!!! note "Nota" - Si no se pasan valores a la función (cuando se `quantileTimingIf`), [NaN](../../sql-reference/data-types/float.md#data_type-float-nan-inf) se devuelve. El propósito de esto es diferenciar estos casos de los casos que resultan en cero. Ver [ORDER BY cláusula](../statements/select/order-by.md#select-order-by) para notas sobre la clasificación `NaN` valor. - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─response_time─┐ -│ 72 │ -│ 112 │ -│ 126 │ -│ 145 │ -│ 104 │ -│ 242 │ -│ 313 │ -│ 168 │ -│ 108 │ -└───────────────┘ -``` - -Consulta: - -``` sql -SELECT quantileTiming(response_time) FROM t -``` - -Resultado: - -``` text -┌─quantileTiming(response_time)─┐ -│ 126 │ -└───────────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileTimingWeighted {#quantiletimingweighted} - -Con la precisión determinada calcula el [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos según el peso de cada miembro de secuencia. - -El resultado es determinista (no depende del orden de procesamiento de la consulta). La función está optimizada para trabajar con secuencias que describen distribuciones como tiempos de carga de páginas web o tiempos de respuesta de back-end. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileTimingWeighted(level)(expr, weight) -``` - -Apodo: `medianTimingWeighted`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). - -- `expr` — [Expresion](../syntax.md#syntax-expressions) sobre una columna valores que devuelven un [Flotante\*](../../sql-reference/data-types/float.md)-tipo número. - - - If negative values are passed to the function, the behavior is undefined. - - If the value is greater than 30,000 (a page loading time of more than 30 seconds), it is assumed to be 30,000. - -- `weight` — Column with weights of sequence elements. Weight is a number of value occurrences. - -**Exactitud** - -El cálculo es preciso si: - -- El número total de valores no supera los 5670. -- El número total de valores supera los 5670, pero el tiempo de carga de la página es inferior a 1024 ms. - -De lo contrario, el resultado del cálculo se redondea al múltiplo más cercano de 16 ms. - -!!! note "Nota" - Para calcular los cuantiles de tiempo de carga de la página, esta función es más efectiva y precisa que [cuantil](#quantile). - -**Valor devuelto** - -- Cuantil del nivel especificado. - -Tipo: `Float32`. - -!!! note "Nota" - Si no se pasan valores a la función (cuando se `quantileTimingIf`), [NaN](../../sql-reference/data-types/float.md#data_type-float-nan-inf) se devuelve. El propósito de esto es diferenciar estos casos de los casos que resultan en cero. Ver [ORDER BY cláusula](../statements/select/order-by.md#select-order-by) para notas sobre la clasificación `NaN` valor. - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─response_time─┬─weight─┐ -│ 68 │ 1 │ -│ 104 │ 2 │ -│ 112 │ 3 │ -│ 126 │ 2 │ -│ 138 │ 1 │ -│ 162 │ 1 │ -└───────────────┴────────┘ -``` - -Consulta: - -``` sql -SELECT quantileTimingWeighted(response_time, weight) FROM t -``` - -Resultado: - -``` text -┌─quantileTimingWeighted(response_time, weight)─┐ -│ 112 │ -└───────────────────────────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileTDigest {#quantiletdigest} - -Calcula un aproximado [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos usando el [T-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algoritmo. - -El error máximo es 1%. El consumo de memoria es `log(n)`, donde `n` es un número de valores. El resultado depende del orden de ejecución de la consulta y no es determinista. - -El rendimiento de la función es menor que el rendimiento de [cuantil](#quantile) o [quantileTiming](#quantiletiming). En términos de la relación entre el tamaño del estado y la precisión, esta función es mucho mejor que `quantile`. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileTDigest(level)(expr) -``` - -Apodo: `medianTDigest`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). - -**Valor devuelto** - -- Cuantil aproximado del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Consulta: - -``` sql -SELECT quantileTDigest(number) FROM numbers(10) -``` - -Resultado: - -``` text -┌─quantileTDigest(number)─┐ -│ 4.5 │ -└─────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## quantileTDigestWeighted {#quantiletdigestweighted} - -Calcula un aproximado [cuantil](https://en.wikipedia.org/wiki/Quantile) de una secuencia de datos numéricos usando el [T-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algoritmo. La función tiene en cuenta el peso de cada miembro de secuencia. El error máximo es 1%. El consumo de memoria es `log(n)`, donde `n` es un número de valores. - -El rendimiento de la función es menor que el rendimiento de [cuantil](#quantile) o [quantileTiming](#quantiletiming). En términos de la relación entre el tamaño del estado y la precisión, esta función es mucho mejor que `quantile`. - -El resultado depende del orden de ejecución de la consulta y no es determinista. - -Cuando se utilizan múltiples `quantile*` funciones con diferentes niveles en una consulta, los estados internos no se combinan (es decir, la consulta funciona de manera menos eficiente de lo que podría). En este caso, use el [cantiles](#quantiles) función. - -**Sintaxis** - -``` sql -quantileTDigest(level)(expr) -``` - -Apodo: `medianTDigest`. - -**Parámetros** - -- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` valor en el rango de `[0.01, 0.99]`. Valor predeterminado: 0.5. En `level=0.5` la función calcula [mediana](https://en.wikipedia.org/wiki/Median). -- `expr` — Expression over the column values resulting in numeric [tipos de datos](../../sql-reference/data-types/index.md#data_types), [Fecha](../../sql-reference/data-types/date.md) o [FechaHora](../../sql-reference/data-types/datetime.md). -- `weight` — Column with weights of sequence elements. Weight is a number of value occurrences. - -**Valor devuelto** - -- Cuantil aproximado del nivel especificado. - -Tipo: - -- [Float64](../../sql-reference/data-types/float.md) para la entrada de tipo de datos numéricos. -- [Fecha](../../sql-reference/data-types/date.md) si los valores de entrada tienen `Date` tipo. -- [FechaHora](../../sql-reference/data-types/datetime.md) si los valores de entrada tienen `DateTime` tipo. - -**Ejemplo** - -Consulta: - -``` sql -SELECT quantileTDigestWeighted(number, 1) FROM numbers(10) -``` - -Resultado: - -``` text -┌─quantileTDigestWeighted(number, 1)─┐ -│ 4.5 │ -└────────────────────────────────────┘ -``` - -**Ver también** - -- [mediana](#median) -- [cantiles](#quantiles) - -## mediana {#median} - -El `median*` funciones son los alias para el `quantile*` función. Calculan la mediana de una muestra de datos numéricos. - -Función: - -- `median` — Alias for [cuantil](#quantile). -- `medianDeterministic` — Alias for [quantileDeterminista](#quantiledeterministic). -- `medianExact` — Alias for [quantileExact](#quantileexact). -- `medianExactWeighted` — Alias for [quantileExactWeighted](#quantileexactweighted). -- `medianTiming` — Alias for [quantileTiming](#quantiletiming). -- `medianTimingWeighted` — Alias for [quantileTimingWeighted](#quantiletimingweighted). -- `medianTDigest` — Alias for [quantileTDigest](#quantiletdigest). -- `medianTDigestWeighted` — Alias for [quantileTDigestWeighted](#quantiletdigestweighted). - -**Ejemplo** - -Tabla de entrada: - -``` text -┌─val─┐ -│ 1 │ -│ 1 │ -│ 2 │ -│ 3 │ -└─────┘ -``` - -Consulta: - -``` sql -SELECT medianDeterministic(val, 1) FROM t -``` - -Resultado: - -``` text -┌─medianDeterministic(val, 1)─┐ -│ 1.5 │ -└─────────────────────────────┘ -``` - -## quantiles(level1, level2, …)(x) {#quantiles} - -Todas las funciones de cuantiles también tienen funciones de cuantiles correspondientes: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`. Estas funciones calculan todos los cuantiles de los niveles enumerados en una sola pasada y devuelven una matriz de los valores resultantes. - -## Acerca de Nosotros) {#varsampx} - -Calcula la cantidad `Σ((x - x̅)^2) / (n - 1)`, donde `n` es el tamaño de la muestra y `x̅`es el valor promedio de `x`. - -Representa una estimación imparcial de la varianza de una variable aleatoria si los valores pasados forman su muestra. - -Devoluciones `Float64`. Cuando `n <= 1`, devoluciones `+∞`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `varSampStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## Nombre de la red inalámbrica (SSID):) {#varpopx} - -Calcula la cantidad `Σ((x - x̅)^2) / n`, donde `n` es el tamaño de la muestra y `x̅`es el valor promedio de `x`. - -En otras palabras, dispersión para un conjunto de valores. Devoluciones `Float64`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `varPopStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## Soporte técnico) {#stddevsampx} - -El resultado es igual a la raíz cuadrada de `varSamp(x)`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `stddevSampStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## stddevPop(x) {#stddevpopx} - -El resultado es igual a la raíz cuadrada de `varPop(x)`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `stddevPopStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## topK(N)(x) {#topknx} - -Devuelve una matriz de los valores aproximadamente más frecuentes de la columna especificada. La matriz resultante se ordena en orden descendente de frecuencia aproximada de valores (no por los valores mismos). - -Implementa el [Ahorro de espacio filtrado](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algoritmo para analizar TopK, basado en el algoritmo de reducción y combinación de [Ahorro de espacio paralelo](https://arxiv.org/pdf/1401.0702.pdf). - -``` sql -topK(N)(column) -``` - -Esta función no proporciona un resultado garantizado. En ciertas situaciones, pueden producirse errores y pueden devolver valores frecuentes que no son los valores más frecuentes. - -Recomendamos usar el `N < 10` valor; el rendimiento se reduce con grandes `N` valor. Valor máximo de `N = 65536`. - -**Parámetros** - -- ‘N’ es el número de elementos a devolver. - -Si se omite el parámetro, se utiliza el valor predeterminado 10. - -**Argumento** - -- ' x ' – The value to calculate frequency. - -**Ejemplo** - -Tome el [A tiempo](../../getting-started/example-datasets/ontime.md) conjunto de datos y seleccione los tres valores más frecuentes `AirlineID` columna. - -``` sql -SELECT topK(3)(AirlineID) AS res -FROM ontime -``` - -``` text -┌─res─────────────────┐ -│ [19393,19790,19805] │ -└─────────────────────┘ -``` - -## topKPeso {#topkweighted} - -Similar a `topK` pero toma un argumento adicional de tipo entero - `weight`. Cada valor se contabiliza `weight` veces para el cálculo de la frecuencia. - -**Sintaxis** - -``` sql -topKWeighted(N)(x, weight) -``` - -**Parámetros** - -- `N` — The number of elements to return. - -**Argumento** - -- `x` – The value. -- `weight` — The weight. [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valor devuelto** - -Devuelve una matriz de los valores con la suma aproximada máxima de pesos. - -**Ejemplo** - -Consulta: - -``` sql -SELECT topKWeighted(10)(number, number) FROM numbers(1000) -``` - -Resultado: - -``` text -┌─topKWeighted(10)(number, number)──────────┐ -│ [999,998,997,996,995,994,993,992,991,990] │ -└───────────────────────────────────────────┘ -``` - -## covarSamp(x, y) {#covarsampx-y} - -Calcula el valor de `Σ((x - x̅)(y - y̅)) / (n - 1)`. - -Devuelve Float64. Cuando `n <= 1`, returns +∞. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `covarSampStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## covarPop(x, y) {#covarpopx-y} - -Calcula el valor de `Σ((x - x̅)(y - y̅)) / n`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `covarPopStable` función. Funciona más lento pero proporciona un menor error computacional. - -## corr(x, y) {#corrx-y} - -Calcula el coeficiente de correlación de Pearson: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`. - -!!! note "Nota" - Esta función utiliza un algoritmo numéricamente inestable. Si necesita [estabilidad numérica](https://en.wikipedia.org/wiki/Numerical_stability) en los cálculos, utilice el `corrStable` función. Funciona más lento, pero proporciona un menor error computacional. - -## categoricalInformationValue {#categoricalinformationvalue} - -Calcula el valor de `(P(tag = 1) - P(tag = 0))(log(P(tag = 1)) - log(P(tag = 0)))` para cada categoría. - -``` sql -categoricalInformationValue(category1, category2, ..., tag) -``` - -El resultado indica cómo una característica discreta (categórica `[category1, category2, ...]` contribuir a un modelo de aprendizaje que predice el valor de `tag`. - -## SimpleLinearRegression {#simplelinearregression} - -Realiza una regresión lineal simple (unidimensional). - -``` sql -simpleLinearRegression(x, y) -``` - -Parámetros: - -- `x` — Column with dependent variable values. -- `y` — Column with explanatory variable values. - -Valores devueltos: - -Constante `(a, b)` de la línea resultante `y = a*x + b`. - -**Ejemplos** - -``` sql -SELECT arrayReduce('simpleLinearRegression', [0, 1, 2, 3], [0, 1, 2, 3]) -``` - -``` text -┌─arrayReduce('simpleLinearRegression', [0, 1, 2, 3], [0, 1, 2, 3])─┐ -│ (1,0) │ -└───────────────────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT arrayReduce('simpleLinearRegression', [0, 1, 2, 3], [3, 4, 5, 6]) -``` - -``` text -┌─arrayReduce('simpleLinearRegression', [0, 1, 2, 3], [3, 4, 5, 6])─┐ -│ (1,3) │ -└───────────────────────────────────────────────────────────────────┘ -``` - -## stochasticLinearRegression {#agg_functions-stochasticlinearregression} - -Esta función implementa la regresión lineal estocástica. Admite parámetros personalizados para la tasa de aprendizaje, el coeficiente de regularización L2, el tamaño de mini lote y tiene pocos métodos para actualizar los pesos ([Adán](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Adam) (utilizado por defecto), [SGD simple](https://en.wikipedia.org/wiki/Stochastic_gradient_descent), [Impulso](https://en.wikipedia.org/wiki/Stochastic_gradient_descent#Momentum), [Nesterov](https://mipt.ru/upload/medialibrary/d7e/41-91.pdf)). - -### Parámetros {#agg_functions-stochasticlinearregression-parameters} - -Hay 4 parámetros personalizables. Se pasan a la función secuencialmente, pero no es necesario pasar los cuatro; se usarán valores predeterminados, sin embargo, un buen modelo requirió algún ajuste de parámetros. - -``` text -stochasticLinearRegression(1.0, 1.0, 10, 'SGD') -``` - -1. `learning rate` es el coeficiente en la longitud del paso, cuando se realiza el paso de descenso de gradiente. Una tasa de aprendizaje demasiado grande puede causar pesos infinitos del modelo. El valor predeterminado es `0.00001`. -2. `l2 regularization coefficient` que puede ayudar a prevenir el sobreajuste. El valor predeterminado es `0.1`. -3. `mini-batch size` establece el número de elementos, cuyos gradientes se calcularán y sumarán para realizar un paso de descenso de gradiente. El descenso estocástico puro usa un elemento, sin embargo, tener lotes pequeños (aproximadamente 10 elementos) hace que los pasos de gradiente sean más estables. El valor predeterminado es `15`. -4. `method for updating weights`, son: `Adam` (predeterminada), `SGD`, `Momentum`, `Nesterov`. `Momentum` y `Nesterov` requieren un poco más de cálculos y memoria, sin embargo, resultan útiles en términos de velocidad de convergencia y estabilidad de los métodos de gradiente estocásticos. - -### Uso {#agg_functions-stochasticlinearregression-usage} - -`stochasticLinearRegression` se utiliza en dos pasos: ajustar el modelo y predecir nuevos datos. Para ajustar el modelo y guardar su estado para su uso posterior, utilizamos `-State` combinador, que básicamente guarda el estado (pesos del modelo, etc.). -Para predecir usamos la función [evalMLMethod](../functions/machine-learning-functions.md#machine_learning_methods-evalmlmethod), que toma un estado como argumento, así como características para predecir. - - - -**1.** Accesorio - -Dicha consulta puede ser utilizada. - -``` sql -CREATE TABLE IF NOT EXISTS train_data -( - param1 Float64, - param2 Float64, - target Float64 -) ENGINE = Memory; - -CREATE TABLE your_model ENGINE = Memory AS SELECT -stochasticLinearRegressionState(0.1, 0.0, 5, 'SGD')(target, param1, param2) -AS state FROM train_data; -``` - -Aquí también tenemos que insertar datos en `train_data` tabla. El número de parámetros no es fijo, depende solo del número de argumentos, pasados a `linearRegressionState`. Todos deben ser valores numéricos. -Tenga en cuenta que la columna con valor objetivo (que nos gustaría aprender a predecir) se inserta como primer argumento. - -**2.** Predecir - -Después de guardar un estado en la tabla, podemos usarlo varias veces para la predicción, o incluso fusionarlo con otros estados y crear nuevos modelos aún mejores. - -``` sql -WITH (SELECT state FROM your_model) AS model SELECT -evalMLMethod(model, param1, param2) FROM test_data -``` - -La consulta devolverá una columna de valores predichos. Tenga en cuenta que el primer argumento de `evalMLMethod` ser `AggregateFunctionState` objeto, siguiente son columnas de características. - -`test_data` es una mesa como `train_data` pero puede no contener el valor objetivo. - -### Nota {#agg_functions-stochasticlinearregression-notes} - -1. Para fusionar dos modelos, el usuario puede crear dicha consulta: - `sql SELECT state1 + state2 FROM your_models` - donde `your_models` la tabla contiene ambos modelos. Esta consulta devolverá un nuevo `AggregateFunctionState` objeto. - -2. El usuario puede obtener pesos del modelo creado para sus propios fines sin guardar el modelo si no `-State` combinador se utiliza. - `sql SELECT stochasticLinearRegression(0.01)(target, param1, param2) FROM train_data` - Dicha consulta se ajustará al modelo y devolverá sus pesos: primero son los pesos, que corresponden a los parámetros del modelo, el último es el sesgo. Entonces, en el ejemplo anterior, la consulta devolverá una columna con 3 valores. - -**Ver también** - -- [stochasticLogisticRegression](#agg_functions-stochasticlogisticregression) -- [Diferencia entre regresiones lineales y logísticas](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) - -## stochasticLogisticRegression {#agg_functions-stochasticlogisticregression} - -Esta función implementa la regresión logística estocástica. Se puede usar para problemas de clasificación binaria, admite los mismos parámetros personalizados que stochasticLinearRegression y funciona de la misma manera. - -### Parámetros {#agg_functions-stochasticlogisticregression-parameters} - -Los parámetros son exactamente los mismos que en stochasticLinearRegression: -`learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`. -Para obtener más información, consulte [parámetros](#agg_functions-stochasticlinearregression-parameters). - -``` text -stochasticLogisticRegression(1.0, 1.0, 10, 'SGD') -``` - -1. Accesorio - - - - See the `Fitting` section in the [stochasticLinearRegression](#stochasticlinearregression-usage-fitting) description. - - Predicted labels have to be in \[-1, 1\]. - -1. Predecir - - - - Using saved state we can predict probability of object having label `1`. - - ``` sql - WITH (SELECT state FROM your_model) AS model SELECT - evalMLMethod(model, param1, param2) FROM test_data - ``` - - The query will return a column of probabilities. Note that first argument of `evalMLMethod` is `AggregateFunctionState` object, next are columns of features. - - We can also set a bound of probability, which assigns elements to different labels. - - ``` sql - SELECT ans < 1.1 AND ans > 0.5 FROM - (WITH (SELECT state FROM your_model) AS model SELECT - evalMLMethod(model, param1, param2) AS ans FROM test_data) - ``` - - Then the result will be labels. - - `test_data` is a table like `train_data` but may not contain target value. - -**Ver también** - -- [stochasticLinearRegression](#agg_functions-stochasticlinearregression) -- [Diferencia entre regresiones lineales y logísticas.](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) - -## Método de codificación de datos: {#groupbitmapand} - -Calcula el AND de una columna de mapa de bits, devuelve la cardinalidad del tipo UInt64, si agrega el sufijo -State, luego devuelve [objeto de mapa de bits](../../sql-reference/functions/bitmap-functions.md). - -``` sql -groupBitmapAnd(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `AggregateFunction(groupBitmap, UInt*)` tipo. - -**Valor de retorno** - -Valor de la `UInt64` tipo. - -**Ejemplo** - -``` sql -DROP TABLE IF EXISTS bitmap_column_expr_test2; -CREATE TABLE bitmap_column_expr_test2 -( - tag_id String, - z AggregateFunction(groupBitmap, UInt32) -) -ENGINE = MergeTree -ORDER BY tag_id; - -INSERT INTO bitmap_column_expr_test2 VALUES ('tag1', bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag2', bitmapBuild(cast([6,7,8,9,10,11,12,13,14,15] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag3', bitmapBuild(cast([2,4,6,8,10,12] as Array(UInt32)))); - -SELECT groupBitmapAnd(z) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─groupBitmapAnd(z)─┐ -│ 3 │ -└───────────────────┘ - -SELECT arraySort(bitmapToArray(groupBitmapAndState(z))) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─arraySort(bitmapToArray(groupBitmapAndState(z)))─┐ -│ [6,8,10] │ -└──────────────────────────────────────────────────┘ -``` - -## Método de codificación de datos: {#groupbitmapor} - -Calcula el OR de una columna de mapa de bits, devuelve la cardinalidad del tipo UInt64, si agrega el sufijo -State, luego devuelve [objeto de mapa de bits](../../sql-reference/functions/bitmap-functions.md). Esto es equivalente a `groupBitmapMerge`. - -``` sql -groupBitmapOr(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `AggregateFunction(groupBitmap, UInt*)` tipo. - -**Valor de retorno** - -Valor de la `UInt64` tipo. - -**Ejemplo** - -``` sql -DROP TABLE IF EXISTS bitmap_column_expr_test2; -CREATE TABLE bitmap_column_expr_test2 -( - tag_id String, - z AggregateFunction(groupBitmap, UInt32) -) -ENGINE = MergeTree -ORDER BY tag_id; - -INSERT INTO bitmap_column_expr_test2 VALUES ('tag1', bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag2', bitmapBuild(cast([6,7,8,9,10,11,12,13,14,15] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag3', bitmapBuild(cast([2,4,6,8,10,12] as Array(UInt32)))); - -SELECT groupBitmapOr(z) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─groupBitmapOr(z)─┐ -│ 15 │ -└──────────────────┘ - -SELECT arraySort(bitmapToArray(groupBitmapOrState(z))) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─arraySort(bitmapToArray(groupBitmapOrState(z)))─┐ -│ [1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] │ -└─────────────────────────────────────────────────┘ -``` - -## Método de codificación de datos: {#groupbitmapxor} - -Calcula el XOR de una columna de mapa de bits, devuelve la cardinalidad del tipo UInt64, si agrega el sufijo -State, luego devuelve [objeto de mapa de bits](../../sql-reference/functions/bitmap-functions.md). - -``` sql -groupBitmapOr(expr) -``` - -**Parámetros** - -`expr` – An expression that results in `AggregateFunction(groupBitmap, UInt*)` tipo. - -**Valor de retorno** - -Valor de la `UInt64` tipo. - -**Ejemplo** - -``` sql -DROP TABLE IF EXISTS bitmap_column_expr_test2; -CREATE TABLE bitmap_column_expr_test2 -( - tag_id String, - z AggregateFunction(groupBitmap, UInt32) -) -ENGINE = MergeTree -ORDER BY tag_id; - -INSERT INTO bitmap_column_expr_test2 VALUES ('tag1', bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag2', bitmapBuild(cast([6,7,8,9,10,11,12,13,14,15] as Array(UInt32)))); -INSERT INTO bitmap_column_expr_test2 VALUES ('tag3', bitmapBuild(cast([2,4,6,8,10,12] as Array(UInt32)))); - -SELECT groupBitmapXor(z) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─groupBitmapXor(z)─┐ -│ 10 │ -└───────────────────┘ - -SELECT arraySort(bitmapToArray(groupBitmapXorState(z))) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%'); -┌─arraySort(bitmapToArray(groupBitmapXorState(z)))─┐ -│ [1,3,5,6,8,10,11,13,14,15] │ -└──────────────────────────────────────────────────┘ -``` - -[Artículo Original](https://clickhouse.tech/docs/en/query_language/agg_functions/reference/) diff --git a/docs/es/sql-reference/ansi.md b/docs/es/sql-reference/ansi.md deleted file mode 100644 index 29e2c5b12e9..00000000000 --- a/docs/es/sql-reference/ansi.md +++ /dev/null @@ -1,180 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: ad252bbb4f7e2899c448eb42ecc39ff195c8faa1 -toc_priority: 40 -toc_title: Compatibilidad con ANSI ---- - -# Compatibilidad de SQL ANSI de ClickHouse SQL Dialect {#ansi-sql-compatibility-of-clickhouse-sql-dialect} - -!!! note "Nota" - Este artículo se basa en la Tabla 38, “Feature taxonomy and definition for mandatory features”, Annex F of ISO/IEC CD 9075-2:2013. - -## Diferencias en el comportamiento {#differences-in-behaviour} - -En la tabla siguiente se enumeran los casos en que la característica de consulta funciona en ClickHouse, pero no se comporta como se especifica en ANSI SQL. - -| Feature ID | Nombre de la función | Diferencia | -|------------|----------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------| -| E011 | Tipos de datos numéricos | El literal numérico con punto se interpreta como aproximado (`Float64`) en lugar de exacta (`Decimal`) | -| E051-05 | Los elementos seleccionados pueden ser renombrados | Los cambios de nombre de los elementos tienen un alcance de visibilidad más amplio que solo el resultado SELECT | -| E141-01 | Restricciones NOT NULL | `NOT NULL` está implícito para las columnas de tabla de forma predeterminada | -| E011-04 | Operadores aritméticos | ClickHouse se desborda en lugar de la aritmética comprobada y cambia el tipo de datos de resultado en función de las reglas personalizadas | - -## Estado de la función {#feature-status} - -| Feature ID | Nombre de la función | Estatus | Comentario | -|------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| **E011** | **Tipos de datos numéricos** | **Parcial**{.text-warning} | | -| E011-01 | Tipos de datos INTEGER y SMALLINT | Sí {.text-success} | | -| E011-02 | REAL, DOUBLE PRECISION y FLOAT tipos de datos tipos de datos | Parcial {.text-warning} | `FLOAT()`, `REAL` y `DOUBLE PRECISION` no son compatibles | -| E011-03 | Tipos de datos DECIMAL y NUMERIC | Parcial {.text-warning} | Solo `DECIMAL(p,s)` es compatible, no `NUMERIC` | -| E011-04 | Operadores aritméticos | Sí {.text-success} | | -| E011-05 | Comparación numérica | Sí {.text-success} | | -| E011-06 | Conversión implícita entre los tipos de datos numéricos | No {.text-danger} | ANSI SQL permite la conversión implícita arbitraria entre tipos numéricos, mientras que ClickHouse se basa en funciones que tienen múltiples sobrecargas en lugar de conversión implícita | -| **E021** | **Tipos de cadena de caracteres** | **Parcial**{.text-warning} | | -| E021-01 | Tipo de datos CHARACTER | No {.text-danger} | | -| E021-02 | Tipo de datos CHARACTER VARYING | No {.text-danger} | `String` se comporta de manera similar, pero sin límite de longitud entre paréntesis | -| E021-03 | Literales de caracteres | Parcial {.text-warning} | Sin concatenación automática de literales consecutivos y compatibilidad con el conjunto de caracteres | -| E021-04 | Función CHARACTER_LENGTH | Parcial {.text-warning} | No `USING` clausula | -| E021-05 | Función OCTET_LENGTH | No {.text-danger} | `LENGTH` se comporta de manera similar | -| E021-06 | SUBSTRING | Parcial {.text-warning} | No hay soporte para `SIMILAR` y `ESCAPE` cláusulas, no `SUBSTRING_REGEX` variante | -| E021-07 | Concatenación de caracteres | Parcial {.text-warning} | No `COLLATE` clausula | -| E021-08 | Funciones SUPERIOR e INFERIOR | Sí {.text-success} | | -| E021-09 | Función TRIM | Sí {.text-success} | | -| E021-10 | Conversión implícita entre los tipos de cadena de caracteres de longitud fija y longitud variable | No {.text-danger} | ANSI SQL permite la conversión implícita arbitraria entre tipos de cadena, mientras que ClickHouse se basa en funciones que tienen múltiples sobrecargas en lugar de conversión implícita | -| E021-11 | Función POSITION | Parcial {.text-warning} | No hay soporte para `IN` y `USING` cláusulas, no `POSITION_REGEX` variante | -| E021-12 | Comparación de caracteres | Sí {.text-success} | | -| **E031** | **Identificador** | **Parcial**{.text-warning} | | -| E031-01 | Identificadores delimitados | Parcial {.text-warning} | El soporte literal Unicode es limitado | -| E031-02 | Identificadores de minúsculas | Sí {.text-success} | | -| E031-03 | Trailing subrayado | Sí {.text-success} | | -| **E051** | **Especificación básica de la consulta** | **Parcial**{.text-warning} | | -| E051-01 | SELECT DISTINCT | Sí {.text-success} | | -| E051-02 | Cláusula GROUP BY | Sí {.text-success} | | -| E051-04 | GROUP BY puede contener columnas que no estén en `` | Oui {.text-success} | | -| E051-05 | Les éléments sélectionnés peuvent être renommés | Oui {.text-success} | | -| E051-06 | Clause HAVING | Oui {.text-success} | | -| E051-07 | Qualifié \* dans la liste select | Oui {.text-success} | | -| E051-08 | Nom de corrélation dans la clause FROM | Oui {.text-success} | | -| E051-09 | Renommer les colonnes de la clause FROM | Aucun {.text-danger} | | -| **E061** | **Prédicats de base et conditions de recherche** | **Partiel**{.text-warning} | | -| E061-01 | Prédicat de comparaison | Oui {.text-success} | | -| E061-02 | Entre prédicat | Partiel {.text-warning} | Aucun `SYMMETRIC` et `ASYMMETRIC` clause | -| E061-03 | Dans le prédicat avec la liste des valeurs | Oui {.text-success} | | -| E061-04 | Comme prédicat | Oui {.text-success} | | -| E061-05 | Comme prédicat: clause D'échappement | Aucun {.text-danger} | | -| E061-06 | Prédicat NULL | Oui {.text-success} | | -| E061-07 | Prédicat de comparaison quantifié | Aucun {.text-danger} | | -| E061-08 | Existe prédicat | Aucun {.text-danger} | | -| E061-09 | Sous-requêtes dans le prédicat de comparaison | Oui {.text-success} | | -| E061-11 | Sous-requêtes dans dans le prédicat | Oui {.text-success} | | -| E061-12 | Sous-requêtes dans le prédicat de comparaison quantifiée | Aucun {.text-danger} | | -| E061-13 | Sous-requêtes corrélées | Aucun {.text-danger} | | -| E061-14 | Condition de recherche | Oui {.text-success} | | -| **E071** | **Expressions de requête de base** | **Partiel**{.text-warning} | | -| E071-01 | Opérateur de table distinct UNION | Aucun {.text-danger} | | -| E071-02 | Opérateur de table UNION ALL | Oui {.text-success} | | -| E071-03 | Sauf opérateur de table DISTINCT | Aucun {.text-danger} | | -| E071-05 | Les colonnes combinées via les opérateurs de table n'ont pas besoin d'avoir exactement le même type de données | Oui {.text-success} | | -| E071-06 | Tableau des opérateurs dans les sous-requêtes | Oui {.text-success} | | -| **E081** | **Les privilèges de base** | **Partiel**{.text-warning} | Les travaux en cours | -| **E091** | **Les fonctions de jeu** | **Oui**{.text-success} | | -| E091-01 | AVG | Oui {.text-success} | | -| E091-02 | COUNT | Oui {.text-success} | | -| E091-03 | MAX | Oui {.text-success} | | -| E091-04 | MIN | Oui {.text-success} | | -| E091-05 | SUM | Oui {.text-success} | | -| E091-06 | TOUS les quantificateurs | Aucun {.text-danger} | | -| E091-07 | Quantificateur DISTINCT | Partiel {.text-warning} | Toutes les fonctions d'agrégation ne sont pas prises en charge | -| **E101** | **Manipulation des données de base** | **Partiel**{.text-warning} | | -| E101-01 | Insérer une déclaration | Oui {.text-success} | Remarque: la clé primaire dans ClickHouse n'implique pas `UNIQUE` contrainte | -| E101-03 | Déclaration de mise à jour recherchée | Aucun {.text-danger} | Il y a un `ALTER UPDATE` déclaration pour la modification des données de lot | -| E101-04 | Requête de suppression recherchée | Aucun {.text-danger} | Il y a un `ALTER DELETE` déclaration pour la suppression de données par lots | -| **E111** | **Instruction SELECT à une ligne** | **Aucun**{.text-danger} | | -| **E121** | **Prise en charge du curseur de base** | **Aucun**{.text-danger} | | -| E121-01 | DECLARE CURSOR | Aucun {.text-danger} | | -| E121-02 | Les colonnes ORDER BY n'ont pas besoin d'être dans la liste select | Aucun {.text-danger} | | -| E121-03 | Expressions de valeur dans la clause ORDER BY | Aucun {.text-danger} | | -| E121-04 | Instruction OPEN | Aucun {.text-danger} | | -| E121-06 | Déclaration de mise à jour positionnée | Aucun {.text-danger} | | -| E121-07 | Instruction de suppression positionnée | Aucun {.text-danger} | | -| E121-08 | Déclaration de fermeture | Aucun {.text-danger} | | -| E121-10 | Instruction FETCH: implicite suivant | Aucun {.text-danger} | | -| E121-17 | Avec curseurs HOLD | Aucun {.text-danger} | | -| **E131** | **Support de valeur Null (nulls au lieu de valeurs)** | **Partiel**{.text-warning} | Certaines restrictions s'appliquent | -| **E141** | **Contraintes d'intégrité de base** | **Partiel**{.text-warning} | | -| E141-01 | Contraintes non nulles | Oui {.text-success} | Note: `NOT NULL` est implicite pour les colonnes de table par défaut | -| E141-02 | Contrainte UNIQUE de colonnes non nulles | Aucun {.text-danger} | | -| E141-03 | Contraintes de clé primaire | Aucun {.text-danger} | | -| E141-04 | Contrainte de clé étrangère de base avec la valeur par défaut NO ACTION Pour l'action de suppression référentielle et l'action de mise à jour référentielle | Aucun {.text-danger} | | -| E141-06 | Vérifier la contrainte | Oui {.text-success} | | -| E141-07 | Colonne par défaut | Oui {.text-success} | | -| E141-08 | Non NULL déduit sur la clé primaire | Oui {.text-success} | | -| E141-10 | Les noms dans une clé étrangère peut être spécifié dans n'importe quel ordre | Aucun {.text-danger} | | -| **E151** | **Support de Transaction** | **Aucun**{.text-danger} | | -| E151-01 | COMMIT déclaration | Aucun {.text-danger} | | -| E151-02 | Déclaration de restauration | Aucun {.text-danger} | | -| **E152** | **Instruction de transaction set de base** | **Aucun**{.text-danger} | | -| E152-01 | SET TRANSACTION statement: clause sérialisable de niveau D'isolement | Aucun {.text-danger} | | -| E152-02 | SET TRANSACTION statement: clauses en lecture seule et en lecture écriture | Aucun {.text-danger} | | -| **E153** | **Requêtes pouvant être mises à jour avec des sous requêtes** | **Aucun**{.text-danger} | | -| **E161** | **Commentaires SQL en utilisant le premier Double moins** | **Oui**{.text-success} | | -| **E171** | **Support SQLSTATE** | **Aucun**{.text-danger} | | -| **E182** | **Liaison du langage hôte** | **Aucun**{.text-danger} | | -| **F031** | **Manipulation de schéma de base** | **Partiel**{.text-warning} | | -| F031-01 | Instruction CREATE TABLE pour créer des tables de base persistantes | Partiel {.text-warning} | Aucun `SYSTEM VERSIONING`, `ON COMMIT`, `GLOBAL`, `LOCAL`, `PRESERVE`, `DELETE`, `REF IS`, `WITH OPTIONS`, `UNDER`, `LIKE`, `PERIOD FOR` clauses et aucun support pour les types de données résolus par l'utilisateur | -| F031-02 | Instruction créer une vue | Partiel {.text-warning} | Aucun `RECURSIVE`, `CHECK`, `UNDER`, `WITH OPTIONS` clauses et aucun support pour les types de données résolus par l'utilisateur | -| F031-03 | Déclaration de subvention | Oui {.text-success} | | -| F031-04 | ALTER TABLE statement: ajouter une clause de colonne | Partiel {.text-warning} | Pas de support pour `GENERATED` clause et période de temps du système | -| F031-13 | Instruction DROP TABLE: clause RESTRICT | Aucun {.text-danger} | | -| F031-16 | Instruction DROP VIEW: clause RESTRICT | Aucun {.text-danger} | | -| F031-19 | REVOKE statement: clause RESTRICT | Aucun {.text-danger} | | -| **F041** | **Table jointe de base** | **Partiel**{.text-warning} | | -| F041-01 | INNER join (mais pas nécessairement le mot-clé INNER) | Oui {.text-success} | | -| F041-02 | INTÉRIEURE mot-clé | Oui {.text-success} | | -| F041-03 | LEFT OUTER JOIN | Oui {.text-success} | | -| F041-04 | RIGHT OUTER JOIN | Oui {.text-success} | | -| F041-05 | Les jointures externes peuvent être imbriqués | Oui {.text-success} | | -| F041-07 | La table intérieure dans une jointure extérieure gauche ou droite peut également être utilisée dans une jointure intérieure | Oui {.text-success} | | -| F041-08 | Tous les opérateurs de comparaison sont pris en charge (plutôt que juste =) | Aucun {.text-danger} | | -| **F051** | **Date et heure de base** | **Partiel**{.text-warning} | | -| F051-01 | Type de données de DATE (y compris la prise en charge du littéral de DATE) | Partiel {.text-warning} | Aucun littéral | -| F051-02 | TYPE DE DONNÉES DE TEMPS (y compris la prise en charge du littéral de temps) avec une précision de secondes fractionnaires d'au moins 0 | Aucun {.text-danger} | | -| F051-03 | Type de données D'horodatage (y compris la prise en charge du littéral D'horodatage) avec une précision de secondes fractionnaires d'au moins 0 et 6 | Aucun {.text-danger} | `DateTime64` temps fournit des fonctionnalités similaires | -| F051-04 | Prédicat de comparaison sur les types de données DATE, heure et horodatage | Partiel {.text-warning} | Un seul type de données disponible | -| F051-05 | Distribution explicite entre les types datetime et les types de chaînes de caractères | Oui {.text-success} | | -| F051-06 | CURRENT_DATE | Aucun {.text-danger} | `today()` est similaire | -| F051-07 | LOCALTIME | Aucun {.text-danger} | `now()` est similaire | -| F051-08 | LOCALTIMESTAMP | Aucun {.text-danger} | | -| **F081** | **UNION et sauf dans les vues** | **Partiel**{.text-warning} | | -| **F131** | **Groupées des opérations** | **Partiel**{.text-warning} | | -| F131-01 | WHERE, GROUP BY et ayant des clauses prises en charge dans les requêtes avec des vues groupées | Oui {.text-success} | | -| F131-02 | Plusieurs tables prises en charge dans les requêtes avec des vues groupées | Oui {.text-success} | | -| F131-03 | Définir les fonctions prises en charge dans les requêtes groupées vues | Oui {.text-success} | | -| F131-04 | Sous requêtes avec des clauses GROUP BY et HAVING et des vues groupées | Oui {.text-success} | | -| F131-05 | Sélectionnez une seule ligne avec des clauses GROUP BY et HAVING et des vues groupées | Aucun {.text-danger} | | -| **F181** | **Support de module Multiple** | **Aucun**{.text-danger} | | -| **F201** | **Fonction de distribution** | **Oui**{.text-success} | | -| **F221** | **Valeurs par défaut explicites** | **Aucun**{.text-danger} | | -| **F261** | **Expression de cas** | **Oui**{.text-success} | | -| F261-01 | Cas Simple | Oui {.text-success} | | -| F261-02 | Cas recherché | Oui {.text-success} | | -| F261-03 | NULLIF | Oui {.text-success} | | -| F261-04 | COALESCE | Oui {.text-success} | | -| **F311** | **Déclaration de définition de schéma** | **Partiel**{.text-warning} | | -| F311-01 | CREATE SCHEMA | Aucun {.text-danger} | | -| F311-02 | Créer une TABLE pour les tables de base persistantes | Oui {.text-success} | | -| F311-03 | CREATE VIEW | Oui {.text-success} | | -| F311-04 | CREATE VIEW: WITH CHECK OPTION | Aucun {.text-danger} | | -| F311-05 | Déclaration de subvention | Oui {.text-success} | | -| **F471** | **Valeurs de sous-requête scalaire** | **Oui**{.text-success} | | -| **F481** | **Prédicat null étendu** | **Oui**{.text-success} | | -| **F812** | **Base de repérage** | **Aucun**{.text-danger} | | -| **T321** | **Routines SQL-invoked de base** | **Aucun**{.text-danger} | | -| T321-01 | Fonctions définies par l'utilisateur sans surcharge | Aucun {.text-danger} | | -| T321-02 | Procédures stockées définies par l'utilisateur sans surcharge | Aucun {.text-danger} | | -| T321-03 | L'invocation de la fonction | Aucun {.text-danger} | | -| T321-04 | L'instruction d'APPEL de | Aucun {.text-danger} | | -| T321-05 | Déclaration de retour | Aucun {.text-danger} | | -| **T631** | **Dans le prédicat avec un élément de liste** | **Oui**{.text-success} | | diff --git a/docs/fr/sql-reference/data-types/aggregatefunction.md b/docs/fr/sql-reference/data-types/aggregatefunction.md deleted file mode 100644 index 18874cd3cb7..00000000000 --- a/docs/fr/sql-reference/data-types/aggregatefunction.md +++ /dev/null @@ -1,70 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 52 -toc_title: AggregateFunction (nom, types_of_arguments...) ---- - -# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} - -Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [une vue matérialisée](../../sql-reference/statements/create.md#create-view). La manière courante de produire un État de fonction d'agrégat est d'appeler la fonction d'agrégat avec le `-State` suffixe. Pour obtenir le résultat final de l'agrégation dans l'avenir, vous devez utiliser la même fonction d'agrégation avec la `-Merge`suffixe. - -`AggregateFunction` — parametric data type. - -**Paramètre** - -- Nom de la fonction d'agrégation. - - If the function is parametric, specify its parameters too. - -- Types des arguments de la fonction d'agrégation. - -**Exemple** - -``` sql -CREATE TABLE t -( - column1 AggregateFunction(uniq, UInt64), - column2 AggregateFunction(anyIf, String, UInt8), - column3 AggregateFunction(quantiles(0.5, 0.9), UInt64) -) ENGINE = ... -``` - -[uniq](../../sql-reference/aggregate-functions/reference.md#agg_function-uniq), anyIf ([tout](../../sql-reference/aggregate-functions/reference.md#agg_function-any)+[Si](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-if)) et [les quantiles](../../sql-reference/aggregate-functions/reference.md) les fonctions d'agrégation sont-elles prises en charge dans ClickHouse. - -## Utilisation {#usage} - -### Insertion De Données {#data-insertion} - -Pour insérer des données, utilisez `INSERT SELECT` avec le regroupement d' `-State`- fonction. - -**Exemples de fonction** - -``` sql -uniqState(UserID) -quantilesState(0.5, 0.9)(SendTiming) -``` - -Contrairement aux fonctions correspondantes `uniq` et `quantiles`, `-State`- les fonctions renvoient l'état, au lieu de la valeur finale. En d'autres termes, ils renvoient une valeur de `AggregateFunction` type. - -Dans les résultats de `SELECT` requête, les valeurs de `AggregateFunction` type ont une représentation binaire spécifique à l'implémentation pour tous les formats de sortie ClickHouse. Si les données de vidage dans, par exemple, `TabSeparated` format avec `SELECT` requête, puis ce vidage peut être chargé en utilisant `INSERT` requête. - -### Sélection De Données {#data-selection} - -Lors de la sélection des données `AggregatingMergeTree` table, utilisez `GROUP BY` et les mêmes fonctions d'agrégat que lors de l'insertion de données, mais en utilisant `-Merge`suffixe. - -Une fonction d'agrégation avec `-Merge` suffixe prend un ensemble d'états, les combine, et renvoie le résultat complet de l'agrégation de données. - -Par exemple, les deux requêtes suivantes retournent le même résultat: - -``` sql -SELECT uniq(UserID) FROM table - -SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID) -``` - -## Exemple D'Utilisation {#usage-example} - -Voir [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) Description du moteur. - -[Article Original](https://clickhouse.tech/docs/en/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/fr/sql-reference/data-types/array.md b/docs/fr/sql-reference/data-types/array.md deleted file mode 100644 index 41772cab177..00000000000 --- a/docs/fr/sql-reference/data-types/array.md +++ /dev/null @@ -1,77 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 51 -toc_title: Array(T) ---- - -# Array(t) {#data-type-array} - -Un tableau de `T`les éléments de type. `T` peut être n'importe quel type de données, y compris un tableau. - -## La création d'un Tableau {#creating-an-array} - -Vous pouvez utiliser une fonction pour créer un tableau: - -``` sql -array(T) -``` - -Vous pouvez également utiliser des crochets. - -``` sql -[] -``` - -Exemple de création d'un tableau: - -``` sql -SELECT array(1, 2) AS x, toTypeName(x) -``` - -``` text -┌─x─────┬─toTypeName(array(1, 2))─┐ -│ [1,2] │ Array(UInt8) │ -└───────┴─────────────────────────┘ -``` - -``` sql -SELECT [1, 2] AS x, toTypeName(x) -``` - -``` text -┌─x─────┬─toTypeName([1, 2])─┐ -│ [1,2] │ Array(UInt8) │ -└───────┴────────────────────┘ -``` - -## Utilisation de Types de données {#working-with-data-types} - -Lors de la création d'un tableau à la volée, ClickHouse définit automatiquement le type d'argument comme le type de données le plus étroit pouvant stocker tous les arguments listés. S'il y a des [Nullable](nullable.md#data_type-nullable) ou littéral [NULL](../../sql-reference/syntax.md#null-literal) les valeurs, le type d'un élément de tableau devient également [Nullable](nullable.md). - -Si ClickHouse n'a pas pu déterminer le type de données, il génère une exception. Par exemple, cela se produit lorsque vous essayez de créer un tableau avec des chaînes et des nombres simultanément (`SELECT array(1, 'a')`). - -Exemples de détection automatique de type de données: - -``` sql -SELECT array(1, 2, NULL) AS x, toTypeName(x) -``` - -``` text -┌─x──────────┬─toTypeName(array(1, 2, NULL))─┐ -│ [1,2,NULL] │ Array(Nullable(UInt8)) │ -└────────────┴───────────────────────────────┘ -``` - -Si vous essayez de créer un tableau de types de données incompatibles, ClickHouse lève une exception: - -``` sql -SELECT array(1, 'a') -``` - -``` text -Received exception from server (version 1.1.54388): -Code: 386. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: There is no supertype for types UInt8, String because some of them are String/FixedString and some of them are not. -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/array/) diff --git a/docs/fr/sql-reference/data-types/boolean.md b/docs/fr/sql-reference/data-types/boolean.md deleted file mode 100644 index aeb84cf1cc1..00000000000 --- a/docs/fr/sql-reference/data-types/boolean.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 43 -toc_title: "Bool\xE9en" ---- - -# Les Valeurs Booléennes {#boolean-values} - -Il n'y a pas de type distinct pour les valeurs booléennes. Utilisez le type UInt8, limité aux valeurs 0 ou 1. - -[Article Original](https://clickhouse.tech/docs/en/data_types/boolean/) diff --git a/docs/fr/sql-reference/data-types/date.md b/docs/fr/sql-reference/data-types/date.md deleted file mode 100644 index 698639f1d2f..00000000000 --- a/docs/fr/sql-reference/data-types/date.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 47 -toc_title: Date ---- - -# Date {#date} - -Date. Stocké en deux octets comme le nombre de jours depuis 1970-01-01 (non signé). Permet de stocker des valeurs juste après le début de L'époque Unix jusqu'au seuil supérieur défini par une constante au stade de la compilation (actuellement, c'est jusqu'à l'année 2106, mais l'année finale entièrement prise en charge est 2105). - -La valeur de date est stockée sans le fuseau horaire. - -[Article Original](https://clickhouse.tech/docs/en/data_types/date/) diff --git a/docs/fr/sql-reference/data-types/datetime.md b/docs/fr/sql-reference/data-types/datetime.md deleted file mode 100644 index 915270e4d2b..00000000000 --- a/docs/fr/sql-reference/data-types/datetime.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 48 -toc_title: DateTime ---- - -# Datetime {#data_type-datetime} - -Permet de stocker un instant dans le temps, qui peut être exprimé comme une date de calendrier et une heure d'une journée. - -Syntaxe: - -``` sql -DateTime([timezone]) -``` - -Plage de valeurs prise en charge: \[1970-01-01 00:00:00, 2105-12-31 23:59:59\]. - -Résolution: 1 seconde. - -## Utilisation Remarques {#usage-remarks} - -Le point dans le temps est enregistré en tant que [Le timestamp Unix](https://en.wikipedia.org/wiki/Unix_time), quel que soit le fuseau horaire ou l'heure d'été. En outre, l' `DateTime` type peut stocker le fuseau horaire qui est le même pour la colonne entière, qui affecte la façon dont les valeurs de la `DateTime` les valeurs de type sont affichées au format texte et comment les valeurs spécifiées en tant que chaînes sont analysées (‘2020-01-01 05:00:01’). Le fuseau horaire n'est pas stocké dans les lignes de la table (ou dans resultset), mais est stocké dans les métadonnées de la colonne. -Une liste des fuseaux horaires pris en charge peut être trouvée dans le [Base de données de fuseau horaire IANA](https://www.iana.org/time-zones). -Le `tzdata` paquet, contenant [Base de données de fuseau horaire IANA](https://www.iana.org/time-zones), doit être installé dans le système. L'utilisation de la `timedatectl list-timezones` commande pour lister les fuseaux horaires connus par un système local. - -Vous pouvez définir explicitement un fuseau horaire `DateTime`- tapez des colonnes lors de la création d'une table. Si le fuseau horaire n'est pas défini, ClickHouse utilise la valeur [fuseau](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) paramètre dans les paramètres du serveur ou les paramètres du système d'exploitation au moment du démarrage du serveur ClickHouse. - -Le [clickhouse-client](../../interfaces/cli.md) applique le fuseau horaire du serveur par défaut si un fuseau horaire n'est pas explicitement défini lors de l'initialisation du type de données. Pour utiliser le fuseau horaire du client, exécutez `clickhouse-client` avec l' `--use_client_time_zone` paramètre. - -Clickhouse affiche les valeurs dans `YYYY-MM-DD hh:mm:ss` format de texte par défaut. Vous pouvez modifier la sortie avec le [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) fonction. - -Lorsque vous insérez des données dans ClickHouse, vous pouvez utiliser différents formats de chaînes de date et d'heure, en fonction de la valeur du [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) paramètre. - -## Exemple {#examples} - -**1.** Création d'une table avec un `DateTime`- tapez la colonne et insérez des données dedans: - -``` sql -CREATE TABLE dt -( - `timestamp` DateTime('Europe/Moscow'), - `event_id` UInt8 -) -ENGINE = TinyLog; -``` - -``` sql -INSERT INTO dt Values (1546300800, 1), ('2019-01-01 00:00:00', 2); -``` - -``` sql -SELECT * FROM dt; -``` - -``` text -┌───────────timestamp─┬─event_id─┐ -│ 2019-01-01 03:00:00 │ 1 │ -│ 2019-01-01 00:00:00 │ 2 │ -└─────────────────────┴──────────┘ -``` - -- Lors de l'insertion de datetime en tant qu'entier, il est traité comme un horodatage Unix (UTC). `1546300800` représenter `'2019-01-01 00:00:00'` L'UTC. Cependant, comme `timestamp` la colonne a `Europe/Moscow` (UTC+3) fuseau horaire spécifié, lors de la sortie en tant que chaîne, la valeur sera affichée comme `'2019-01-01 03:00:00'` -- Lors de l'insertion d'une valeur de chaîne en tant que datetime, elle est traitée comme étant dans le fuseau horaire de la colonne. `'2019-01-01 00:00:00'` sera considérée comme étant en `Europe/Moscow` fuseau horaire et enregistré sous `1546290000`. - -**2.** Le filtrage sur `DateTime` valeur - -``` sql -SELECT * FROM dt WHERE timestamp = toDateTime('2019-01-01 00:00:00', 'Europe/Moscow') -``` - -``` text -┌───────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00 │ 2 │ -└─────────────────────┴──────────┘ -``` - -`DateTime` les valeurs de colonne peuvent être filtrées à l'aide d'une `WHERE` prédicat. Elle sera convertie `DateTime` automatiquement: - -``` sql -SELECT * FROM dt WHERE timestamp = '2019-01-01 00:00:00' -``` - -``` text -┌───────────timestamp─┬─event_id─┐ -│ 2019-01-01 03:00:00 │ 1 │ -└─────────────────────┴──────────┘ -``` - -**3.** Obtenir un fuseau horaire pour un `DateTime`colonne de type: - -``` sql -SELECT toDateTime(now(), 'Europe/Moscow') AS column, toTypeName(column) AS x -``` - -``` text -┌──────────────column─┬─x─────────────────────────┐ -│ 2019-10-16 04:12:04 │ DateTime('Europe/Moscow') │ -└─────────────────────┴───────────────────────────┘ -``` - -**4.** Conversion de fuseau horaire - -``` sql -SELECT -toDateTime(timestamp, 'Europe/London') as lon_time, -toDateTime(timestamp, 'Europe/Moscow') as mos_time -FROM dt -``` - -``` text -┌───────────lon_time──┬────────────mos_time─┐ -│ 2019-01-01 00:00:00 │ 2019-01-01 03:00:00 │ -│ 2018-12-31 21:00:00 │ 2019-01-01 00:00:00 │ -└─────────────────────┴─────────────────────┘ -``` - -## Voir Aussi {#see-also} - -- [Fonctions de conversion de Type](../../sql-reference/functions/type-conversion-functions.md) -- [Fonctions pour travailler avec des dates et des heures](../../sql-reference/functions/date-time-functions.md) -- [Fonctions pour travailler avec des tableaux](../../sql-reference/functions/array-functions.md) -- [Le `date_time_input_format` paramètre](../../operations/settings/settings.md#settings-date_time_input_format) -- [Le `timezone` paramètre de configuration du serveur](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) -- [Opérateurs pour travailler avec des dates et des heures](../../sql-reference/operators/index.md#operators-datetime) -- [Le `Date` type de données](date.md) - -[Article Original](https://clickhouse.tech/docs/en/data_types/datetime/) diff --git a/docs/fr/sql-reference/data-types/datetime64.md b/docs/fr/sql-reference/data-types/datetime64.md deleted file mode 100644 index 027891c595d..00000000000 --- a/docs/fr/sql-reference/data-types/datetime64.md +++ /dev/null @@ -1,104 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 49 -toc_title: DateTime64 ---- - -# Datetime64 {#data_type-datetime64} - -Permet de stocker un instant dans le temps, qui peut être exprimé comme une date de calendrier et une heure d'un jour, avec une précision de sous-seconde définie - -Tick taille (précision): 10-précision deuxième - -Syntaxe: - -``` sql -DateTime64(precision, [timezone]) -``` - -En interne, stocke les données comme un certain nombre de ‘ticks’ depuis le début de l'époque (1970-01-01 00: 00: 00 UTC) comme Int64. La résolution des tiques est déterminée par le paramètre de précision. En outre, l' `DateTime64` type peut stocker le fuseau horaire qui est le même pour la colonne entière, qui affecte la façon dont les valeurs de la `DateTime64` les valeurs de type sont affichées au format texte et comment les valeurs spécifiées en tant que chaînes sont analysées (‘2020-01-01 05:00:01.000’). Le fuseau horaire n'est pas stocké dans les lignes de la table (ou dans resultset), mais est stocké dans les métadonnées de la colonne. Voir les détails dans [DateTime](datetime.md). - -## Exemple {#examples} - -**1.** Création d'une table avec `DateTime64`- tapez la colonne et insérez des données dedans: - -``` sql -CREATE TABLE dt -( - `timestamp` DateTime64(3, 'Europe/Moscow'), - `event_id` UInt8 -) -ENGINE = TinyLog -``` - -``` sql -INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2) -``` - -``` sql -SELECT * FROM dt -``` - -``` text -┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 03:00:00.000 │ 1 │ -│ 2019-01-01 00:00:00.000 │ 2 │ -└─────────────────────────┴──────────┘ -``` - -- Lors de l'insertion de datetime en tant qu'entier, il est traité comme un horodatage Unix (UTC) mis à l'échelle de manière appropriée. `1546300800000` (avec précision 3) représente `'2019-01-01 00:00:00'` L'UTC. Cependant, comme `timestamp` la colonne a `Europe/Moscow` (UTC+3) fuseau horaire spécifié, lors de la sortie sous forme de chaîne, la valeur sera affichée comme `'2019-01-01 03:00:00'` -- Lors de l'insertion d'une valeur de chaîne en tant que datetime, elle est traitée comme étant dans le fuseau horaire de la colonne. `'2019-01-01 00:00:00'` sera considérée comme étant en `Europe/Moscow` fuseau horaire et stocké comme `1546290000000`. - -**2.** Le filtrage sur `DateTime64` valeur - -``` sql -SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow') -``` - -``` text -┌───────────────timestamp─┬─event_id─┐ -│ 2019-01-01 00:00:00.000 │ 2 │ -└─────────────────────────┴──────────┘ -``` - -Contrairement `DateTime`, `DateTime64` les valeurs ne sont pas converties depuis `String` automatiquement - -**3.** Obtenir un fuseau horaire pour un `DateTime64`-le type de la valeur: - -``` sql -SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x -``` - -``` text -┌──────────────────column─┬─x──────────────────────────────┐ -│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Europe/Moscow') │ -└─────────────────────────┴────────────────────────────────┘ -``` - -**4.** Conversion de fuseau horaire - -``` sql -SELECT -toDateTime64(timestamp, 3, 'Europe/London') as lon_time, -toDateTime64(timestamp, 3, 'Europe/Moscow') as mos_time -FROM dt -``` - -``` text -┌───────────────lon_time──┬────────────────mos_time─┐ -│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │ -│ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │ -└─────────────────────────┴─────────────────────────┘ -``` - -## Voir Aussi {#see-also} - -- [Fonctions de conversion de Type](../../sql-reference/functions/type-conversion-functions.md) -- [Fonctions pour travailler avec des dates et des heures](../../sql-reference/functions/date-time-functions.md) -- [Fonctions pour travailler avec des tableaux](../../sql-reference/functions/array-functions.md) -- [Le `date_time_input_format` paramètre](../../operations/settings/settings.md#settings-date_time_input_format) -- [Le `timezone` paramètre de configuration du serveur](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) -- [Opérateurs pour travailler avec des dates et des heures](../../sql-reference/operators/index.md#operators-datetime) -- [`Date` type de données](date.md) -- [`DateTime` type de données](datetime.md) diff --git a/docs/fr/sql-reference/data-types/decimal.md b/docs/fr/sql-reference/data-types/decimal.md deleted file mode 100644 index 171bc1cf6dd..00000000000 --- a/docs/fr/sql-reference/data-types/decimal.md +++ /dev/null @@ -1,109 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 42 -toc_title: "D\xE9cimal" ---- - -# Décimal (P, S), Décimal32 (S), Décimal64 (S), Décimal128 (S) {#decimalp-s-decimal32s-decimal64s-decimal128s} - -Nombres à points fixes signés qui conservent la précision pendant les opérations d'addition, de soustraction et de multiplication. Pour la division, les chiffres les moins significatifs sont ignorés (non arrondis). - -## Paramètre {#parameters} - -- P-précision. Plage valide: \[1: 38 \]. Détermine le nombre de chiffres décimaux nombre peut avoir (fraction y compris). -- S - échelle. Plage valide: \[0: P \]. Détermine le nombre de chiffres décimaux fraction peut avoir. - -En fonction de P Paramètre Valeur décimal (P, S) est un synonyme de: -- P à partir de \[ 1: 9\] - Pour Décimal32 (S) -- P à partir de \[10: 18\] - pour Décimal64 (S) -- P à partir de \[19: 38\] - pour Décimal128 (S) - -## Plages De Valeurs Décimales {#decimal-value-ranges} - -- Décimal32 (S) - ( -1 \* 10^(9 - S), 1 \* 10^(9-S) ) -- Décimal64 (S) - ( -1 \* 10^(18 - S), 1 \* 10^(18-S) ) -- Décimal128 (S) - ( -1 \* 10^(38 - S), 1 \* 10^(38-S) ) - -Par exemple, Decimal32(4) peut contenir des nombres de -99999.9999 à 99999.9999 avec 0,0001 étape. - -## Représentation Interne {#internal-representation} - -En interne, les données sont représentées comme des entiers signés normaux avec une largeur de bit respective. Les plages de valeurs réelles qui peuvent être stockées en mémoire sont un peu plus grandes que celles spécifiées ci-dessus, qui sont vérifiées uniquement lors de la conversion à partir d'une chaîne. - -Parce que les processeurs modernes ne prennent pas en charge les entiers 128 bits nativement, les opérations sur Decimal128 sont émulées. Pour cette raison, Decimal128 fonctionne significativement plus lentement que Decimal32 / Decimal64. - -## Opérations et type de résultat {#operations-and-result-type} - -Les opérations binaires sur le résultat décimal dans le type de résultat plus large (avec n'importe quel ordre d'arguments). - -- `Decimal64(S1) Decimal32(S2) -> Decimal64(S)` -- `Decimal128(S1) Decimal32(S2) -> Decimal128(S)` -- `Decimal128(S1) Decimal64(S2) -> Decimal128(S)` - -Règles pour l'échelle: - -- ajouter, soustraire: S = max (S1, S2). -- multuply: S = S1 + S2. -- diviser: S = S1. - -Pour des opérations similaires entre décimal et entier, le résultat est Décimal de la même taille qu'un argument. - -Les opérations entre Decimal et Float32 / Float64 ne sont pas définies. Si vous en avez besoin, vous pouvez explicitement lancer l'un des arguments en utilisant les builtins toDecimal32, toDecimal64, toDecimal128 ou toFloat32, toFloat64. Gardez à l'esprit que le résultat perdra de la précision et que la conversion de type est une opération coûteuse en calcul. - -Certaines fonctions sur le résultat de retour décimal comme Float64 (par exemple, var ou stddev). Les calculs intermédiaires peuvent toujours être effectués en décimal, ce qui peut conduire à des résultats différents entre les entrées Float64 et Decimal avec les mêmes valeurs. - -## Contrôles De Débordement {#overflow-checks} - -Pendant les calculs sur Décimal, des débordements entiers peuvent se produire. Les chiffres excessifs dans une fraction sont éliminés (non arrondis). Les chiffres excessifs dans la partie entière conduiront à une exception. - -``` sql -SELECT toDecimal32(2, 4) AS x, x / 3 -``` - -``` text -┌──────x─┬─divide(toDecimal32(2, 4), 3)─┐ -│ 2.0000 │ 0.6666 │ -└────────┴──────────────────────────────┘ -``` - -``` sql -SELECT toDecimal32(4.2, 8) AS x, x * x -``` - -``` text -DB::Exception: Scale is out of bounds. -``` - -``` sql -SELECT toDecimal32(4.2, 8) AS x, 6 * x -``` - -``` text -DB::Exception: Decimal math overflow. -``` - -Les contrôles de débordement entraînent un ralentissement des opérations. S'il est connu que les débordements ne sont pas possibles, il est logique de désactiver les contrôles en utilisant `decimal_check_overflow` paramètre. Lorsque des contrôles sont désactivés et le débordement se produit, le résultat sera faux: - -``` sql -SET decimal_check_overflow = 0; -SELECT toDecimal32(4.2, 8) AS x, 6 * x -``` - -``` text -┌──────────x─┬─multiply(6, toDecimal32(4.2, 8))─┐ -│ 4.20000000 │ -17.74967296 │ -└────────────┴──────────────────────────────────┘ -``` - -Les contrôles de débordement se produisent non seulement sur les opérations arithmétiques mais aussi sur la comparaison de valeurs: - -``` sql -SELECT toDecimal32(1, 8) < 100 -``` - -``` text -DB::Exception: Can't compare. -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/decimal/) diff --git a/docs/fr/sql-reference/data-types/domains/index.md b/docs/fr/sql-reference/data-types/domains/index.md deleted file mode 100644 index 7e11f9a8a68..00000000000 --- a/docs/fr/sql-reference/data-types/domains/index.md +++ /dev/null @@ -1,33 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Domaine -toc_priority: 56 -toc_title: "Aper\xE7u" ---- - -# Domaine {#domains} - -Les domaines sont des types spéciaux qui ajoutent des fonctionnalités supplémentaires au sommet du type de base existant, mais en laissant le format on-wire et on-disc du type de données sous-jacent intact. À l'heure actuelle, ClickHouse ne prend pas en charge les domaines définis par l'utilisateur. - -Vous pouvez utiliser des domaines partout type de base correspondant peut être utilisé, par exemple: - -- Créer une colonne d'un type de domaine -- Valeurs de lecture / écriture depuis / vers la colonne de domaine -- L'utiliser comme un indice si un type de base peut être utilisée comme un indice -- Fonctions d'appel avec des valeurs de colonne de domaine - -### Fonctionnalités supplémentaires des domaines {#extra-features-of-domains} - -- Nom de type de colonne explicite dans `SHOW CREATE TABLE` ou `DESCRIBE TABLE` -- Entrée du format convivial avec `INSERT INTO domain_table(domain_column) VALUES(...)` -- Sortie au format convivial pour `SELECT domain_column FROM domain_table` -- Chargement de données à partir d'une source externe dans un format convivial: `INSERT INTO domain_table FORMAT CSV ...` - -### Limitation {#limitations} - -- Impossible de convertir la colonne d'index du type de base en type de domaine via `ALTER TABLE`. -- Impossible de convertir implicitement des valeurs de chaîne en valeurs de domaine lors de l'insertion de données d'une autre colonne ou table. -- Le domaine n'ajoute aucune contrainte sur les valeurs stockées. - -[Article Original](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/fr/sql-reference/data-types/domains/ipv4.md b/docs/fr/sql-reference/data-types/domains/ipv4.md deleted file mode 100644 index 12895992e77..00000000000 --- a/docs/fr/sql-reference/data-types/domains/ipv4.md +++ /dev/null @@ -1,84 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 59 -toc_title: IPv4 ---- - -## IPv4 {#ipv4} - -`IPv4` est un domaine basé sur `UInt32` tapez et sert de remplacement typé pour stocker des valeurs IPv4. Il fournit un stockage compact avec le format d'entrée-sortie convivial et les informations de type de colonne sur l'inspection. - -### Utilisation De Base {#basic-usage} - -``` sql -CREATE TABLE hits (url String, from IPv4) ENGINE = MergeTree() ORDER BY url; - -DESCRIBE TABLE hits; -``` - -``` text -┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┐ -│ url │ String │ │ │ │ │ -│ from │ IPv4 │ │ │ │ │ -└──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┘ -``` - -Ou vous pouvez utiliser le domaine IPv4 comme clé: - -``` sql -CREATE TABLE hits (url String, from IPv4) ENGINE = MergeTree() ORDER BY from; -``` - -`IPv4` le domaine prend en charge le format d'entrée personnalisé en tant que chaînes IPv4: - -``` sql -INSERT INTO hits (url, from) VALUES ('https://wikipedia.org', '116.253.40.133')('https://clickhouse.tech', '183.247.232.58')('https://clickhouse.tech/docs/en/', '116.106.34.242'); - -SELECT * FROM hits; -``` - -``` text -┌─url────────────────────────────────┬───────────from─┐ -│ https://clickhouse.tech/docs/en/ │ 116.106.34.242 │ -│ https://wikipedia.org │ 116.253.40.133 │ -│ https://clickhouse.tech │ 183.247.232.58 │ -└────────────────────────────────────┴────────────────┘ -``` - -Les valeurs sont stockées sous forme binaire compacte: - -``` sql -SELECT toTypeName(from), hex(from) FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(from)─┬─hex(from)─┐ -│ IPv4 │ B7F7E83A │ -└──────────────────┴───────────┘ -``` - -Les valeurs de domaine ne sont pas implicitement convertibles en types autres que `UInt32`. -Si vous voulez convertir `IPv4` valeur à une chaîne, vous devez le faire explicitement avec `IPv4NumToString()` fonction: - -``` sql -SELECT toTypeName(s), IPv4NumToString(from) as s FROM hits LIMIT 1; -``` - - ┌─toTypeName(IPv4NumToString(from))─┬─s──────────────┐ - │ String │ 183.247.232.58 │ - └───────────────────────────────────┴────────────────┘ - -Ou coulé à un `UInt32` valeur: - -``` sql -SELECT toTypeName(i), CAST(from as UInt32) as i FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(CAST(from, 'UInt32'))─┬──────────i─┐ -│ UInt32 │ 3086477370 │ -└──────────────────────────────────┴────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/domains/ipv4) diff --git a/docs/fr/sql-reference/data-types/domains/ipv6.md b/docs/fr/sql-reference/data-types/domains/ipv6.md deleted file mode 100644 index 77510a950cb..00000000000 --- a/docs/fr/sql-reference/data-types/domains/ipv6.md +++ /dev/null @@ -1,86 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 60 -toc_title: IPv6 ---- - -## IPv6 {#ipv6} - -`IPv6` est un domaine basé sur `FixedString(16)` tapez et sert de remplacement typé pour stocker des valeurs IPv6. Il fournit un stockage compact avec le format d'entrée-sortie convivial et les informations de type de colonne sur l'inspection. - -### Utilisation De Base {#basic-usage} - -``` sql -CREATE TABLE hits (url String, from IPv6) ENGINE = MergeTree() ORDER BY url; - -DESCRIBE TABLE hits; -``` - -``` text -┌─name─┬─type───┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┐ -│ url │ String │ │ │ │ │ -│ from │ IPv6 │ │ │ │ │ -└──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┘ -``` - -Ou vous pouvez utiliser `IPv6` domaine comme l'un des principaux: - -``` sql -CREATE TABLE hits (url String, from IPv6) ENGINE = MergeTree() ORDER BY from; -``` - -`IPv6` le domaine prend en charge l'entrée personnalisée en tant que chaînes IPv6: - -``` sql -INSERT INTO hits (url, from) VALUES ('https://wikipedia.org', '2a02:aa08:e000:3100::2')('https://clickhouse.tech', '2001:44c8:129:2632:33:0:252:2')('https://clickhouse.tech/docs/en/', '2a02:e980:1e::1'); - -SELECT * FROM hits; -``` - -``` text -┌─url────────────────────────────────┬─from──────────────────────────┐ -│ https://clickhouse.tech │ 2001:44c8:129:2632:33:0:252:2 │ -│ https://clickhouse.tech/docs/en/ │ 2a02:e980:1e::1 │ -│ https://wikipedia.org │ 2a02:aa08:e000:3100::2 │ -└────────────────────────────────────┴───────────────────────────────┘ -``` - -Les valeurs sont stockées sous forme binaire compacte: - -``` sql -SELECT toTypeName(from), hex(from) FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(from)─┬─hex(from)────────────────────────┐ -│ IPv6 │ 200144C8012926320033000002520002 │ -└──────────────────┴──────────────────────────────────┘ -``` - -Les valeurs de domaine ne sont pas implicitement convertibles en types autres que `FixedString(16)`. -Si vous voulez convertir `IPv6` valeur à une chaîne, vous devez le faire explicitement avec `IPv6NumToString()` fonction: - -``` sql -SELECT toTypeName(s), IPv6NumToString(from) as s FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(IPv6NumToString(from))─┬─s─────────────────────────────┐ -│ String │ 2001:44c8:129:2632:33:0:252:2 │ -└───────────────────────────────────┴───────────────────────────────┘ -``` - -Ou coulé à un `FixedString(16)` valeur: - -``` sql -SELECT toTypeName(i), CAST(from as FixedString(16)) as i FROM hits LIMIT 1; -``` - -``` text -┌─toTypeName(CAST(from, 'FixedString(16)'))─┬─i───────┐ -│ FixedString(16) │ ��� │ -└───────────────────────────────────────────┴─────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/domains/ipv6) diff --git a/docs/fr/sql-reference/data-types/enum.md b/docs/fr/sql-reference/data-types/enum.md deleted file mode 100644 index b9751c1c804..00000000000 --- a/docs/fr/sql-reference/data-types/enum.md +++ /dev/null @@ -1,132 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 50 -toc_title: Enum ---- - -# Enum {#enum} - -Type énuméré composé de valeurs nommées. - -Les valeurs nommées doivent être déclarées comme `'string' = integer` pair. ClickHouse ne stocke que des nombres, mais prend en charge les opérations avec les valeurs à travers leurs noms. - -Supports ClickHouse: - -- 8-bit `Enum`. Il peut contenir jusqu'à 256 valeurs énumérées dans le `[-128, 127]` gamme. -- 16 bits `Enum`. Il peut contenir jusqu'à 65 536 valeurs énumérées dans le `[-32768, 32767]` gamme. - -Clickhouse choisit automatiquement le type de `Enum` lorsque les données sont insérées. Vous pouvez également utiliser `Enum8` ou `Enum16` types pour être sûr de la taille de stockage. - -## Exemples D'Utilisation {#usage-examples} - -Ici, nous créons une table avec une `Enum8('hello' = 1, 'world' = 2)` type de colonne: - -``` sql -CREATE TABLE t_enum -( - x Enum('hello' = 1, 'world' = 2) -) -ENGINE = TinyLog -``` - -Colonne `x` ne peut stocker que les valeurs répertoriées dans la définition de type: `'hello'` ou `'world'`. Si vous essayez d'enregistrer une autre valeur, ClickHouse déclenchera une exception. Taille 8 bits pour cela `Enum` est choisi automatiquement. - -``` sql -INSERT INTO t_enum VALUES ('hello'), ('world'), ('hello') -``` - -``` text -Ok. -``` - -``` sql -INSERT INTO t_enum values('a') -``` - -``` text -Exception on client: -Code: 49. DB::Exception: Unknown element 'a' for type Enum('hello' = 1, 'world' = 2) -``` - -Lorsque vous interrogez des données de la table, ClickHouse affiche les valeurs de chaîne de `Enum`. - -``` sql -SELECT * FROM t_enum -``` - -``` text -┌─x─────┐ -│ hello │ -│ world │ -│ hello │ -└───────┘ -``` - -Si vous avez besoin de voir les équivalents numériques des lignes, vous devez `Enum` valeur en type entier. - -``` sql -SELECT CAST(x, 'Int8') FROM t_enum -``` - -``` text -┌─CAST(x, 'Int8')─┐ -│ 1 │ -│ 2 │ -│ 1 │ -└─────────────────┘ -``` - -Pour créer une valeur d'Enum dans une requête, vous devez également utiliser `CAST`. - -``` sql -SELECT toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)')) -``` - -``` text -┌─toTypeName(CAST('a', 'Enum(\'a\' = 1, \'b\' = 2)'))─┐ -│ Enum8('a' = 1, 'b' = 2) │ -└─────────────────────────────────────────────────────┘ -``` - -## Règles générales et utilisation {#general-rules-and-usage} - -Chacune des valeurs se voit attribuer un nombre dans la plage `-128 ... 127` pour `Enum8` ou dans la gamme `-32768 ... 32767` pour `Enum16`. Toutes les chaînes et les nombres doivent être différents. Une chaîne vide est autorisé. Si ce type est spécifié (dans une définition de table), les nombres peuvent être dans un ordre arbitraire. Toutefois, l'ordre n'a pas d'importance. - -Ni la chaîne ni la valeur numérique dans un `Enum` peut être [NULL](../../sql-reference/syntax.md). - -Un `Enum` peut être contenue dans [Nullable](nullable.md) type. Donc, si vous créez une table en utilisant la requête - -``` sql -CREATE TABLE t_enum_nullable -( - x Nullable( Enum8('hello' = 1, 'world' = 2) ) -) -ENGINE = TinyLog -``` - -il peut stocker non seulement des `'hello'` et `'world'`, mais `NULL`, ainsi. - -``` sql -INSERT INTO t_enum_nullable Values('hello'),('world'),(NULL) -``` - -Dans la mémoire RAM, un `Enum` la colonne est stockée dans la même manière que `Int8` ou `Int16` des valeurs numériques correspondantes. - -Lors de la lecture sous forme de texte, ClickHouse analyse la valeur sous forme de chaîne et recherche la chaîne correspondante à partir de l'ensemble des valeurs Enum. Si elle n'est pas trouvée, une exception est levée. Lors de la lecture au format texte, la chaîne est lue et la valeur numérique correspondante est recherchée. Une exception sera levée si il n'est pas trouvé. -Lors de l'écriture sous forme de texte, il écrit la valeur correspondante de la chaîne. Si les données de colonne contiennent des déchets (nombres qui ne proviennent pas de l'ensemble valide), une exception est levée. Lors de la lecture et de l'écriture sous forme binaire, cela fonctionne de la même manière que pour les types de données Int8 et Int16. -La valeur implicite par défaut est la valeur avec le numéro le plus bas. - -Lors `ORDER BY`, `GROUP BY`, `IN`, `DISTINCT` et ainsi de suite, les Énumérations se comportent de la même façon que les nombres correspondants. Par exemple, ORDER BY les trie numériquement. Les opérateurs d'égalité et de comparaison fonctionnent de la même manière sur les énumérations que sur les valeurs numériques sous-jacentes. - -Les valeurs Enum ne peuvent pas être comparées aux nombres. Les Enums peuvent être comparés à une chaîne constante. Si la chaîne comparée à n'est pas une valeur valide pour L'énumération, une exception sera levée. L'opérateur est pris en charge avec l'Enum sur le côté gauche, et un ensemble de chaînes sur le côté droit. Les chaînes sont les valeurs de L'énumération correspondante. - -Most numeric and string operations are not defined for Enum values, e.g. adding a number to an Enum or concatenating a string to an Enum. -Cependant, L'énumération a un naturel `toString` fonction qui renvoie sa valeur de chaîne. - -Les valeurs Enum sont également convertibles en types numériques en utilisant `toT` fonction, où T est un type numérique. Lorsque T correspond au type numérique sous-jacent de l'énumération, cette conversion est à coût nul. -Le type Enum peut être modifié sans coût en utilisant ALTER, si seulement l'ensemble des valeurs est modifié. Il est possible d'ajouter et de supprimer des membres de L'énumération en utilisant ALTER (la suppression n'est sûre que si la valeur supprimée n'a jamais été utilisée dans la table). À titre de sauvegarde, la modification de la valeur numérique d'un membre Enum précédemment défini lancera une exception. - -En utilisant ALTER, il est possible de changer un Enum8 en Enum16 ou vice versa, tout comme changer un Int8 en Int16. - -[Article Original](https://clickhouse.tech/docs/en/data_types/enum/) diff --git a/docs/fr/sql-reference/data-types/fixedstring.md b/docs/fr/sql-reference/data-types/fixedstring.md deleted file mode 100644 index 5ba09187581..00000000000 --- a/docs/fr/sql-reference/data-types/fixedstring.md +++ /dev/null @@ -1,63 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: FixedString (N) ---- - -# Fixedstring {#fixedstring} - -Une chaîne de longueur fixe de `N` octets (ni caractères ni points de code). - -Pour déclarer une colonne de `FixedString` tapez, utilisez la syntaxe suivante: - -``` sql - FixedString(N) -``` - -Où `N` est un nombre naturel. - -Le `FixedString` type est efficace lorsque les données ont la longueur de précisément `N` octet. Dans tous les autres cas, il est susceptible de réduire l'efficacité. - -Exemples de valeurs qui peuvent être stockées efficacement dans `FixedString`-tapé colonnes: - -- La représentation binaire des adresses IP (`FixedString(16)` pour IPv6). -- Language codes (ru_RU, en_US … ). -- Currency codes (USD, RUB … ). -- Représentation binaire des hachages (`FixedString(16)` pour MD5, `FixedString(32)` pour SHA256). - -Pour stocker les valeurs UUID, utilisez [UUID](uuid.md) type de données. - -Lors de l'insertion des données, ClickHouse: - -- Complète une chaîne avec des octets null si la chaîne contient moins de `N` octet. -- Jette le `Too large value for FixedString(N)` exception si la chaîne contient plus de `N` octet. - -Lors de la sélection des données, ClickHouse ne supprime pas les octets nuls à la fin de la chaîne. Si vous utilisez le `WHERE` clause, vous devez ajouter des octets null manuellement pour `FixedString` valeur. L'exemple suivant illustre l'utilisation de l' `WHERE` la clause de `FixedString`. - -Considérons le tableau suivant avec le seul `FixedString(2)` colonne: - -``` text -┌─name──┐ -│ b │ -└───────┘ -``` - -Requête `SELECT * FROM FixedStringTable WHERE a = 'b'` ne renvoie aucune donnée en conséquence. Nous devrions compléter le modèle de filtre avec des octets nuls. - -``` sql -SELECT * FROM FixedStringTable -WHERE a = 'b\0' -``` - -``` text -┌─a─┐ -│ b │ -└───┘ -``` - -Ce comportement diffère de MySQL pour le `CHAR` type (où les chaînes sont remplies d'espaces et les espaces sont supprimés pour la sortie). - -À noter que la longueur de la `FixedString(N)` la valeur est constante. Le [longueur](../../sql-reference/functions/array-functions.md#array_functions-length) la fonction renvoie `N` même si l' `FixedString(N)` la valeur est remplie uniquement avec des octets [vide](../../sql-reference/functions/string-functions.md#empty) la fonction renvoie `1` dans ce cas. - -[Article Original](https://clickhouse.tech/docs/en/data_types/fixedstring/) diff --git a/docs/fr/sql-reference/data-types/float.md b/docs/fr/sql-reference/data-types/float.md deleted file mode 100644 index b269b930110..00000000000 --- a/docs/fr/sql-reference/data-types/float.md +++ /dev/null @@ -1,87 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: Float32, Float64 ---- - -# Float32, Float64 {#float32-float64} - -[Les nombres à virgule flottante](https://en.wikipedia.org/wiki/IEEE_754). - -Les Types sont équivalents aux types de C: - -- `Float32` - `float` -- `Float64` - `double` - -Nous vous recommandons de stocker les données sous forme entière chaque fois que possible. Par exemple, convertissez des nombres de précision fixes en valeurs entières, telles que des montants monétaires ou des temps de chargement de page en millisecondes. - -## Utilisation de nombres à virgule flottante {#using-floating-point-numbers} - -- Calculs avec des nombres à virgule flottante peut produire une erreur d'arrondi. - - - -``` sql -SELECT 1 - 0.9 -``` - -``` text -┌───────minus(1, 0.9)─┐ -│ 0.09999999999999998 │ -└─────────────────────┘ -``` - -- Le résultat du calcul dépend de la méthode de calcul (le type de processeur et de l'architecture du système informatique). -- Les calculs à virgule flottante peuvent entraîner des nombres tels que l'infini (`Inf`) et “not-a-number” (`NaN`). Cela doit être pris en compte lors du traitement des résultats de calculs. -- Lors de l'analyse de nombres à virgule flottante à partir de texte, le résultat peut ne pas être le nombre représentable par machine le plus proche. - -## NaN et Inf {#data_type-float-nan-inf} - -Contrairement à SQL standard, ClickHouse prend en charge les catégories suivantes de nombres à virgule flottante: - -- `Inf` – Infinity. - - - -``` sql -SELECT 0.5 / 0 -``` - -``` text -┌─divide(0.5, 0)─┐ -│ inf │ -└────────────────┘ -``` - -- `-Inf` – Negative infinity. - - - -``` sql -SELECT -0.5 / 0 -``` - -``` text -┌─divide(-0.5, 0)─┐ -│ -inf │ -└─────────────────┘ -``` - -- `NaN` – Not a number. - - - -``` sql -SELECT 0 / 0 -``` - -``` text -┌─divide(0, 0)─┐ -│ nan │ -└──────────────┘ -``` - - See the rules for `NaN` sorting in the section [ORDER BY clause](../sql_reference/statements/select/order-by.md). - -[Article Original](https://clickhouse.tech/docs/en/data_types/float/) diff --git a/docs/fr/sql-reference/data-types/index.md b/docs/fr/sql-reference/data-types/index.md deleted file mode 100644 index 887e2efd69f..00000000000 --- a/docs/fr/sql-reference/data-types/index.md +++ /dev/null @@ -1,15 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Types De Donn\xE9es" -toc_priority: 37 -toc_title: Introduction ---- - -# Types De Données {#data_types} - -ClickHouse peut stocker différents types de données dans des cellules de table. - -Cette section décrit les types de données pris en charge et les considérations spéciales pour les utiliser et/ou les implémenter le cas échéant. - -[Article Original](https://clickhouse.tech/docs/en/data_types/) diff --git a/docs/fr/sql-reference/data-types/int-uint.md b/docs/fr/sql-reference/data-types/int-uint.md deleted file mode 100644 index 9b196c164a4..00000000000 --- a/docs/fr/sql-reference/data-types/int-uint.md +++ /dev/null @@ -1,26 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 ---- - -# UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} - -Entiers de longueur fixe, avec ou sans signe. - -## Plages Int {#int-ranges} - -- Int8 - \[-128: 127\] -- Int16 - \[-32768: 32767\] -- Int32 - \[-2147483648: 2147483647\] -- Int64 - \[-9223372036854775808: 9223372036854775807\] - -## Plages Uint {#uint-ranges} - -- UInt8 - \[0: 255\] -- UInt16 - \[0: 65535\] -- UInt32- \[0: 4294967295\] -- UInt64- \[0: 18446744073709551615\] - -[Article Original](https://clickhouse.tech/docs/en/data_types/int_uint/) diff --git a/docs/fr/sql-reference/data-types/nested-data-structures/index.md b/docs/fr/sql-reference/data-types/nested-data-structures/index.md deleted file mode 100644 index 528e0bad0cd..00000000000 --- a/docs/fr/sql-reference/data-types/nested-data-structures/index.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Structures De Donn\xE9es Imbriqu\xE9es" -toc_hidden: true -toc_priority: 54 -toc_title: "cach\xE9s" ---- - -# Structures De Données Imbriquées {#nested-data-structures} - -[Article Original](https://clickhouse.tech/docs/en/data_types/nested_data_structures/) diff --git a/docs/fr/sql-reference/data-types/nested-data-structures/nested.md b/docs/fr/sql-reference/data-types/nested-data-structures/nested.md deleted file mode 100644 index 2805780de24..00000000000 --- a/docs/fr/sql-reference/data-types/nested-data-structures/nested.md +++ /dev/null @@ -1,106 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 57 -toc_title: "Imbriqu\xE9e(Type1 Nom1, Nom2 Type2, ...)" ---- - -# Nested(name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2} - -A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create.md) requête. Chaque ligne de table peut correspondre à n'importe quel nombre de lignes dans une structure de données imbriquée. - -Exemple: - -``` sql -CREATE TABLE test.visits -( - CounterID UInt32, - StartDate Date, - Sign Int8, - IsNew UInt8, - VisitID UInt64, - UserID UInt64, - ... - Goals Nested - ( - ID UInt32, - Serial UInt32, - EventTime DateTime, - Price Int64, - OrderID String, - CurrencyID UInt32 - ), - ... -) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign) -``` - -Cet exemple déclare le `Goals` structure de données imbriquée, qui contient des données sur les conversions (objectifs atteints). Chaque ligne de la ‘visits’ table peut correspondre à zéro ou n'importe quel nombre de conversions. - -Un seul niveau d'imbrication est pris en charge. Les colonnes de structures imbriquées contenant des tableaux sont équivalentes à des tableaux multidimensionnels, elles ont donc un support limité (il n'y a pas de support pour stocker ces colonnes dans des tables avec le moteur MergeTree). - -Dans la plupart des cas, lorsque vous travaillez avec une structure de données imbriquée, ses colonnes sont spécifiées avec des noms de colonnes séparés par un point. Ces colonnes constituent un tableau de types correspondants. Tous les tableaux de colonnes d'une structure de données imbriquée unique ont la même longueur. - -Exemple: - -``` sql -SELECT - Goals.ID, - Goals.EventTime -FROM test.visits -WHERE CounterID = 101500 AND length(Goals.ID) < 5 -LIMIT 10 -``` - -``` text -┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐ -│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │ -│ [1073752] │ ['2014-03-17 00:28:25'] │ -│ [1073752] │ ['2014-03-17 10:46:20'] │ -│ [1073752,591325,591325,591325] │ ['2014-03-17 13:59:20','2014-03-17 22:17:55','2014-03-17 22:18:07','2014-03-17 22:18:51'] │ -│ [] │ [] │ -│ [1073752,591325,591325] │ ['2014-03-17 11:37:06','2014-03-17 14:07:47','2014-03-17 14:36:21'] │ -│ [] │ [] │ -│ [] │ [] │ -│ [591325,1073752] │ ['2014-03-17 00:46:05','2014-03-17 00:46:05'] │ -│ [1073752,591325,591325,591325] │ ['2014-03-17 13:28:33','2014-03-17 13:30:26','2014-03-17 18:51:21','2014-03-17 18:51:45'] │ -└────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -Il est plus facile de penser à une structure de données imbriquée comme un ensemble de plusieurs tableaux de colonnes de la même longueur. - -Le seul endroit où une requête SELECT peut spécifier le nom d'une structure de données imbriquée entière au lieu de colonnes individuelles est la clause de jointure de tableau. Pour plus d'informations, voir “ARRAY JOIN clause”. Exemple: - -``` sql -SELECT - Goal.ID, - Goal.EventTime -FROM test.visits -ARRAY JOIN Goals AS Goal -WHERE CounterID = 101500 AND length(Goals.ID) < 5 -LIMIT 10 -``` - -``` text -┌─Goal.ID─┬──────Goal.EventTime─┐ -│ 1073752 │ 2014-03-17 16:38:10 │ -│ 591325 │ 2014-03-17 16:38:48 │ -│ 591325 │ 2014-03-17 16:42:27 │ -│ 1073752 │ 2014-03-17 00:28:25 │ -│ 1073752 │ 2014-03-17 10:46:20 │ -│ 1073752 │ 2014-03-17 13:59:20 │ -│ 591325 │ 2014-03-17 22:17:55 │ -│ 591325 │ 2014-03-17 22:18:07 │ -│ 591325 │ 2014-03-17 22:18:51 │ -│ 1073752 │ 2014-03-17 11:37:06 │ -└─────────┴─────────────────────┘ -``` - -Vous ne pouvez pas effectuer SELECT pour une structure de données imbriquée entière. Vous ne pouvez lister explicitement que les colonnes individuelles qui en font partie. - -Pour une requête INSERT, vous devez passer tous les tableaux de colonnes composant d'une structure de données imbriquée séparément (comme s'il s'agissait de tableaux de colonnes individuels). Au cours de l'insertion, le système vérifie qu'ils ont la même longueur. - -Pour une requête DESCRIBE, les colonnes d'une structure de données imbriquée sont répertoriées séparément de la même manière. - -La requête ALTER pour les éléments d'une structure de données imbriquée a des limites. - -[Article Original](https://clickhouse.tech/docs/en/data_types/nested_data_structures/nested/) diff --git a/docs/fr/sql-reference/data-types/nullable.md b/docs/fr/sql-reference/data-types/nullable.md deleted file mode 100644 index 6b37b571a96..00000000000 --- a/docs/fr/sql-reference/data-types/nullable.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 54 -toc_title: Nullable ---- - -# Nullable(typename) {#data_type-nullable} - -Permet de stocker marqueur spécial ([NULL](../../sql-reference/syntax.md)) qui dénote “missing value” aux valeurs normales autorisées par `TypeName`. Par exemple, un `Nullable(Int8)` type colonne peut stocker `Int8` type de valeurs, et les lignes qui n'ont pas de valeur magasin `NULL`. - -Pour un `TypeName` vous ne pouvez pas utiliser les types de données composites [Tableau](array.md) et [Tuple](tuple.md). Les types de données composites peuvent contenir `Nullable` valeurs de type, telles que `Array(Nullable(Int8))`. - -A `Nullable` le champ type ne peut pas être inclus dans les index de table. - -`NULL` est la valeur par défaut pour tout `Nullable` type, sauf indication contraire dans la configuration du serveur ClickHouse. - -## Caractéristiques De Stockage {#storage-features} - -Stocker `Nullable` valeurs de type dans une colonne de table, ClickHouse utilise un fichier séparé avec `NULL` masques en plus du fichier normal avec des valeurs. Les entrées du fichier masks permettent à ClickHouse de faire la distinction entre `NULL` et une valeur par défaut du type de données correspondant pour chaque ligne de table. En raison d'un fichier supplémentaire, `Nullable` colonne consomme de l'espace de stockage supplémentaire par rapport à une normale similaire. - -!!! info "Note" - Utiliser `Nullable` affecte presque toujours négativement les performances, gardez cela à l'esprit lors de la conception de vos bases de données. - -## Exemple D'Utilisation {#usage-example} - -``` sql -CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog -``` - -``` sql -INSERT INTO t_null VALUES (1, NULL), (2, 3) -``` - -``` sql -SELECT x + y FROM t_null -``` - -``` text -┌─plus(x, y)─┐ -│ ᴺᵁᴸᴸ │ -│ 5 │ -└────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/nullable/) diff --git a/docs/fr/sql-reference/data-types/simpleaggregatefunction.md b/docs/fr/sql-reference/data-types/simpleaggregatefunction.md deleted file mode 100644 index 81fcd67cfae..00000000000 --- a/docs/fr/sql-reference/data-types/simpleaggregatefunction.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# SimpleAggregateFunction {#data-type-simpleaggregatefunction} - -`SimpleAggregateFunction(name, types_of_arguments…)` le type de données stocke la valeur actuelle de la fonction d'agrégat et ne stocke pas son état complet comme [`AggregateFunction`](aggregatefunction.md) faire. Cette optimisation peut être appliquée aux fonctions pour lesquelles la propriété suivante est conservée: le résultat de l'application d'une fonction `f` pour un ensemble de lignes `S1 UNION ALL S2` peut être obtenu en appliquant `f` pour les parties de la ligne définie séparément, puis à nouveau l'application `f` pour les résultats: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Cette propriété garantit que les résultats d'agrégation partielle sont suffisants pour calculer le combiné, de sorte que nous n'avons pas à stocker et traiter de données supplémentaires. - -Les fonctions d'agrégation suivantes sont prises en charge: - -- [`any`](../../sql-reference/aggregate-functions/reference.md#agg_function-any) -- [`anyLast`](../../sql-reference/aggregate-functions/reference.md#anylastx) -- [`min`](../../sql-reference/aggregate-functions/reference.md#agg_function-min) -- [`max`](../../sql-reference/aggregate-functions/reference.md#agg_function-max) -- [`sum`](../../sql-reference/aggregate-functions/reference.md#agg_function-sum) -- [`groupBitAnd`](../../sql-reference/aggregate-functions/reference.md#groupbitand) -- [`groupBitOr`](../../sql-reference/aggregate-functions/reference.md#groupbitor) -- [`groupBitXor`](../../sql-reference/aggregate-functions/reference.md#groupbitxor) - -Les valeurs de la `SimpleAggregateFunction(func, Type)` regarder et stockées de la même manière que `Type`, de sorte que vous n'avez pas besoin d'appliquer des fonctions avec `-Merge`/`-State` suffixe. `SimpleAggregateFunction` a de meilleures performances que `AggregateFunction` avec la même fonction d'agrégation. - -**Paramètre** - -- Nom de la fonction d'agrégation. -- Types des arguments de la fonction d'agrégation. - -**Exemple** - -``` sql -CREATE TABLE t -( - column1 SimpleAggregateFunction(sum, UInt64), - column2 SimpleAggregateFunction(any, String) -) ENGINE = ... -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) diff --git a/docs/fr/sql-reference/data-types/special-data-types/expression.md b/docs/fr/sql-reference/data-types/special-data-types/expression.md deleted file mode 100644 index c3ba5e42ba1..00000000000 --- a/docs/fr/sql-reference/data-types/special-data-types/expression.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: Expression ---- - -# Expression {#expression} - -Les Expressions sont utilisées pour représenter des lambdas dans des fonctions d'ordre Élevé. - -[Article Original](https://clickhouse.tech/docs/en/data_types/special_data_types/expression/) diff --git a/docs/fr/sql-reference/data-types/special-data-types/index.md b/docs/fr/sql-reference/data-types/special-data-types/index.md deleted file mode 100644 index 6d292dc522e..00000000000 --- a/docs/fr/sql-reference/data-types/special-data-types/index.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "Types De Donn\xE9es Sp\xE9ciaux" -toc_hidden: true -toc_priority: 55 -toc_title: "cach\xE9s" ---- - -# Types De Données Spéciaux {#special-data-types} - -Les valeurs de type de données spéciales ne peuvent pas être sérialisées pour l'enregistrement dans une table ou la sortie dans les résultats de la requête, mais peuvent être utilisées comme résultat intermédiaire lors de l'exécution de la requête. - -[Article Original](https://clickhouse.tech/docs/en/data_types/special_data_types/) diff --git a/docs/fr/sql-reference/data-types/special-data-types/interval.md b/docs/fr/sql-reference/data-types/special-data-types/interval.md deleted file mode 100644 index 464de8a10ab..00000000000 --- a/docs/fr/sql-reference/data-types/special-data-types/interval.md +++ /dev/null @@ -1,85 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 61 -toc_title: Intervalle ---- - -# Intervalle {#data-type-interval} - -Famille de types de données représentant des intervalles d'heure et de date. Les types de la [INTERVAL](../../../sql-reference/operators/index.md#operator-interval) opérateur. - -!!! warning "Avertissement" - `Interval` les valeurs de type de données ne peuvent pas être stockées dans les tables. - -Structure: - -- Intervalle de temps en tant que valeur entière non signée. -- Type de l'intervalle. - -Types d'intervalles pris en charge: - -- `SECOND` -- `MINUTE` -- `HOUR` -- `DAY` -- `WEEK` -- `MONTH` -- `QUARTER` -- `YEAR` - -Pour chaque type d'intervalle, il existe un type de données distinct. Par exemple, l' `DAY` l'intervalle correspond au `IntervalDay` type de données: - -``` sql -SELECT toTypeName(INTERVAL 4 DAY) -``` - -``` text -┌─toTypeName(toIntervalDay(4))─┐ -│ IntervalDay │ -└──────────────────────────────┘ -``` - -## Utilisation Remarques {#data-type-interval-usage-remarks} - -Vous pouvez utiliser `Interval`-tapez des valeurs dans des opérations arithmétiques avec [Date](../../../sql-reference/data-types/date.md) et [DateTime](../../../sql-reference/data-types/datetime.md)-type de valeurs. Par exemple, vous pouvez ajouter 4 jours à l'heure actuelle: - -``` sql -SELECT now() as current_date_time, current_date_time + INTERVAL 4 DAY -``` - -``` text -┌───current_date_time─┬─plus(now(), toIntervalDay(4))─┐ -│ 2019-10-23 10:58:45 │ 2019-10-27 10:58:45 │ -└─────────────────────┴───────────────────────────────┘ -``` - -Les intervalles avec différents types ne peuvent pas être combinés. Vous ne pouvez pas utiliser des intervalles comme `4 DAY 1 HOUR`. Spécifiez des intervalles en unités inférieures ou égales à la plus petite unité de l'intervalle, par exemple, l'intervalle `1 day and an hour` l'intervalle peut être exprimée comme `25 HOUR` ou `90000 SECOND`. - -Vous ne pouvez pas effectuer d'opérations arithmétiques avec `Interval`- tapez des valeurs, mais vous pouvez ajouter des intervalles de différents types par conséquent aux valeurs dans `Date` ou `DateTime` types de données. Exemple: - -``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR -``` - -``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ -``` - -La requête suivante provoque une exception: - -``` sql -select now() AS current_date_time, current_date_time + (INTERVAL 4 DAY + INTERVAL 3 HOUR) -``` - -``` text -Received exception from server (version 19.14.1): -Code: 43. DB::Exception: Received from localhost:9000. DB::Exception: Wrong argument types for function plus: if one argument is Interval, then another must be Date or DateTime.. -``` - -## Voir Aussi {#see-also} - -- [INTERVAL](../../../sql-reference/operators/index.md#operator-interval) opérateur -- [toInterval](../../../sql-reference/functions/type-conversion-functions.md#function-tointerval) type fonctions de conversion diff --git a/docs/fr/sql-reference/data-types/special-data-types/nothing.md b/docs/fr/sql-reference/data-types/special-data-types/nothing.md deleted file mode 100644 index 2e3d76b7207..00000000000 --- a/docs/fr/sql-reference/data-types/special-data-types/nothing.md +++ /dev/null @@ -1,26 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 60 -toc_title: Rien ---- - -# Rien {#nothing} - -Le seul but de ce type de données est de représenter les cas où une valeur n'est pas prévu. Donc vous ne pouvez pas créer un `Nothing` type de valeur. - -Par exemple, littéral [NULL](../../../sql-reference/syntax.md#null-literal) a type de `Nullable(Nothing)`. Voir plus sur [Nullable](../../../sql-reference/data-types/nullable.md). - -Le `Nothing` type peut également être utilisé pour désigner des tableaux vides: - -``` sql -SELECT toTypeName(array()) -``` - -``` text -┌─toTypeName(array())─┐ -│ Array(Nothing) │ -└─────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/special_data_types/nothing/) diff --git a/docs/fr/sql-reference/data-types/special-data-types/set.md b/docs/fr/sql-reference/data-types/special-data-types/set.md deleted file mode 100644 index 8f50175bb6b..00000000000 --- a/docs/fr/sql-reference/data-types/special-data-types/set.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 59 -toc_title: "D\xE9finir" ---- - -# Définir {#set} - -Utilisé pour la moitié droite d'un [IN](../../operators/in.md#select-in-operators) expression. - -[Article Original](https://clickhouse.tech/docs/en/data_types/special_data_types/set/) diff --git a/docs/fr/sql-reference/data-types/string.md b/docs/fr/sql-reference/data-types/string.md deleted file mode 100644 index b82e1fe6c69..00000000000 --- a/docs/fr/sql-reference/data-types/string.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: "Cha\xEEne" ---- - -# Chaîne {#string} - -Les chaînes d'une longueur arbitraire. La longueur n'est pas limitée. La valeur peut contenir un ensemble arbitraire d'octets, y compris des octets nuls. -Le type de chaîne remplace les types VARCHAR, BLOB, CLOB et autres provenant d'autres SGBD. - -## Encodage {#encodings} - -ClickHouse n'a pas le concept d'encodages. Les chaînes peuvent contenir un ensemble arbitraire d'octets, qui sont stockés et sortis tels quels. -Si vous avez besoin de stocker des textes, nous vous recommandons d'utiliser L'encodage UTF-8. À tout le moins, si votre terminal utilise UTF-8 (comme recommandé), vous pouvez lire et écrire vos valeurs sans effectuer de conversions. -De même, certaines fonctions pour travailler avec des chaînes ont des variations distinctes qui fonctionnent sous l'hypothèse que la chaîne contient un ensemble d'octets représentant un texte codé en UTF-8. -Par exemple, l' ‘length’ fonction calcule la longueur de la chaîne en octets, tandis que le ‘lengthUTF8’ la fonction calcule la longueur de la chaîne en points de code Unicode, en supposant que la valeur est encodée en UTF-8. - -[Article Original](https://clickhouse.tech/docs/en/data_types/string/) diff --git a/docs/fr/sql-reference/data-types/tuple.md b/docs/fr/sql-reference/data-types/tuple.md deleted file mode 100644 index ab9db735181..00000000000 --- a/docs/fr/sql-reference/data-types/tuple.md +++ /dev/null @@ -1,52 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 53 -toc_title: Tuple (T1, T2,...) ---- - -# Tuple(t1, T2, …) {#tuplet1-t2} - -Un n-uplet d'éléments, chacun ayant une personne [type](index.md#data_types). - -Les Tuples sont utilisés pour le regroupement temporaire de colonnes. Les colonnes peuvent être regroupées lorsqu'une expression IN est utilisée dans une requête et pour spécifier certains paramètres formels des fonctions lambda. Pour plus d'informations, voir les sections [Dans les opérateurs](../../sql-reference/operators/in.md) et [Des fonctions d'ordre supérieur](../../sql-reference/functions/higher-order-functions.md). - -Les Tuples peuvent être le résultat d'une requête. Dans ce cas, pour les formats de texte autres que JSON, les valeurs sont séparées par des virgules entre parenthèses. Dans les formats JSON, les tuples sont sortis sous forme de tableaux (entre crochets). - -## La création d'un Tuple {#creating-a-tuple} - -Vous pouvez utiliser une fonction pour créer un tuple: - -``` sql -tuple(T1, T2, ...) -``` - -Exemple de création d'un tuple: - -``` sql -SELECT tuple(1,'a') AS x, toTypeName(x) -``` - -``` text -┌─x───────┬─toTypeName(tuple(1, 'a'))─┐ -│ (1,'a') │ Tuple(UInt8, String) │ -└─────────┴───────────────────────────┘ -``` - -## Utilisation de Types de données {#working-with-data-types} - -Lors de la création d'un tuple à la volée, ClickHouse détecte automatiquement le type de chaque argument comme le minimum des types qui peuvent stocker la valeur de l'argument. Si l'argument est [NULL](../../sql-reference/syntax.md#null-literal) le type de l'élément tuple est [Nullable](nullable.md). - -Exemple de détection automatique de type de données: - -``` sql -SELECT tuple(1, NULL) AS x, toTypeName(x) -``` - -``` text -┌─x────────┬─toTypeName(tuple(1, NULL))──────┐ -│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │ -└──────────┴─────────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/data_types/tuple/) diff --git a/docs/fr/sql-reference/data-types/uuid.md b/docs/fr/sql-reference/data-types/uuid.md deleted file mode 100644 index 60973a3f855..00000000000 --- a/docs/fr/sql-reference/data-types/uuid.md +++ /dev/null @@ -1,77 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 46 -toc_title: UUID ---- - -# UUID {#uuid-data-type} - -Un identifiant unique universel (UUID) est un numéro de 16 octets utilisé pour identifier les enregistrements. Pour plus d'informations sur L'UUID, voir [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). - -L'exemple de valeur de type UUID est représenté ci-dessous: - -``` text -61f0c404-5cb3-11e7-907b-a6006ad3dba0 -``` - -Si vous ne spécifiez pas la valeur de la colonne UUID lors de l'insertion d'un nouvel enregistrement, la valeur UUID est remplie avec zéro: - -``` text -00000000-0000-0000-0000-000000000000 -``` - -## Comment générer {#how-to-generate} - -Pour générer la valeur UUID, ClickHouse fournit [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) fonction. - -## Exemple D'Utilisation {#usage-example} - -**Exemple 1** - -Cet exemple montre la création d'une table avec la colonne de type UUID et l'insertion d'une valeur dans la table. - -``` sql -CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog -``` - -``` sql -INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1' -``` - -``` sql -SELECT * FROM t_uuid -``` - -``` text -┌────────────────────────────────────x─┬─y─────────┐ -│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │ -└──────────────────────────────────────┴───────────┘ -``` - -**Exemple 2** - -Dans cet exemple, la valeur de la colonne UUID n'est pas spécifiée lors de l'insertion d'un nouvel enregistrement. - -``` sql -INSERT INTO t_uuid (y) VALUES ('Example 2') -``` - -``` sql -SELECT * FROM t_uuid -``` - -``` text -┌────────────────────────────────────x─┬─y─────────┐ -│ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │ -│ 00000000-0000-0000-0000-000000000000 │ Example 2 │ -└──────────────────────────────────────┴───────────┘ -``` - -## Restriction {#restrictions} - -Le type de données UUID ne prend en charge que les fonctions qui [Chaîne](string.md) type de données prend également en charge (par exemple, [min](../../sql-reference/aggregate-functions/reference.md#agg_function-min), [Max](../../sql-reference/aggregate-functions/reference.md#agg_function-max), et [compter](../../sql-reference/aggregate-functions/reference.md#agg_function-count)). - -Le type de données UUID n'est pas pris en charge par les opérations arithmétiques (par exemple, [ABS](../../sql-reference/functions/arithmetic-functions.md#arithm_func-abs)) ou des fonctions d'agrégation, comme [somme](../../sql-reference/aggregate-functions/reference.md#agg_function-sum) et [avg](../../sql-reference/aggregate-functions/reference.md#agg_function-avg). - -[Article Original](https://clickhouse.tech/docs/en/data_types/uuid/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md deleted file mode 100644 index cc238f02f3a..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md +++ /dev/null @@ -1,70 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: "Dictionnaires hi\xE9rarchiques" ---- - -# Dictionnaires Hiérarchiques {#hierarchical-dictionaries} - -Clickhouse prend en charge les dictionnaires hiérarchiques avec un [touche numérique](external-dicts-dict-structure.md#ext_dict-numeric-key). - -Voici une structure hiérarchique: - -``` text -0 (Common parent) -│ -├── 1 (Russia) -│ │ -│ └── 2 (Moscow) -│ │ -│ └── 3 (Center) -│ -└── 4 (Great Britain) - │ - └── 5 (London) -``` - -Cette hiérarchie peut être exprimée comme la table de dictionnaire suivante. - -| id_région | région_parent | nom_région | -|------------|----------------|--------------------| -| 1 | 0 | Russie | -| 2 | 1 | Moscou | -| 3 | 2 | Center | -| 4 | 0 | La Grande-Bretagne | -| 5 | 4 | Londres | - -Ce tableau contient une colonne `parent_region` qui contient la clé du parent le plus proche de l'élément. - -Clickhouse soutient le [hiérarchique](external-dicts-dict-structure.md#hierarchical-dict-attr) propriété pour [externe dictionnaire](index.md) attribut. Cette propriété vous permet de configurer le dictionnaire hiérarchique comme décrit ci-dessus. - -Le [dictGetHierarchy](../../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy) la fonction vous permet d'obtenir la chaîne parent d'un élément. - -Pour notre exemple, la structure du dictionnaire peut être la suivante: - -``` xml - - - - region_id - - - - parent_region - UInt64 - 0 - true - - - - region_name - String - - - - - -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_hierarchical/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md deleted file mode 100644 index 2569329fefd..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ /dev/null @@ -1,407 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: "Stockage des dictionnaires en m\xE9moire" ---- - -# Stockage des dictionnaires en mémoire {#dicts-external-dicts-dict-layout} - -Il existe une variété de façons de stocker les dictionnaires en mémoire. - -Nous vous recommandons [plat](#flat), [haché](#dicts-external_dicts_dict_layout-hashed) et [complex_key_hashed](#complex-key-hashed). qui fournissent la vitesse de traitement optimale. - -La mise en cache n'est pas recommandée en raison de performances potentiellement médiocres et de difficultés à sélectionner les paramètres optimaux. En savoir plus dans la section “[cache](#cache)”. - -Il existe plusieurs façons d'améliorer les performances du dictionnaire: - -- Appelez la fonction pour travailler avec le dictionnaire après `GROUP BY`. -- Marquer les attributs à extraire comme injectifs. Un attribut est appelé injectif si différentes valeurs d'attribut correspondent à différentes clés. Alors, quand `GROUP BY` utilise une fonction qui récupère une valeur d'attribut par la clé, cette fonction est automatiquement retirée de `GROUP BY`. - -ClickHouse génère une exception pour les erreurs avec les dictionnaires. Des exemples d'erreurs: - -- Le dictionnaire accessible n'a pas pu être chargé. -- Erreur de la requête d'une `cached` dictionnaire. - -Vous pouvez afficher la liste des dictionnaires externes et leurs statuts dans le `system.dictionaries` table. - -La configuration ressemble à ceci: - -``` xml - - - ... - - - - - - ... - - -``` - -Correspondant [DDL-requête](../../statements/create.md#create-dictionary-query): - -``` sql -CREATE DICTIONARY (...) -... -LAYOUT(LAYOUT_TYPE(param value)) -- layout settings -... -``` - -## Façons de stocker des dictionnaires en mémoire {#ways-to-store-dictionaries-in-memory} - -- [plat](#flat) -- [haché](#dicts-external_dicts_dict_layout-hashed) -- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) -- [cache](#cache) -- [direct](#direct) -- [range_hashed](#range-hashed) -- [complex_key_hashed](#complex-key-hashed) -- [complex_key_cache](#complex-key-cache) -- [complex_key_direct](#complex-key-direct) -- [ip_trie](#ip-trie) - -### plat {#flat} - -Le dictionnaire est complètement stocké en mémoire sous la forme de tableaux plats. Combien de mémoire le dictionnaire utilise-t-il? Le montant est proportionnel à la taille de la plus grande clé (dans l'espace). - -La clé du dictionnaire a le `UInt64` type et la valeur est limitée à 500 000. Si une clé plus grande est découverte lors de la création du dictionnaire, ClickHouse lève une exception et ne crée pas le dictionnaire. - -Tous les types de sources sont pris en charge. Lors de la mise à jour, les données (à partir d'un fichier ou d'une table) sont lues dans leur intégralité. - -Cette méthode fournit les meilleures performances parmi toutes les méthodes disponibles de stockage du dictionnaire. - -Exemple de Configuration: - -``` xml - - - -``` - -ou - -``` sql -LAYOUT(FLAT()) -``` - -### haché {#dicts-external_dicts_dict_layout-hashed} - -Le dictionnaire est entièrement stockée en mémoire sous la forme d'une table de hachage. Le dictionnaire peut contenir n'importe quel nombre d'éléments avec tous les identificateurs Dans la pratique, le nombre de clés peut atteindre des dizaines de millions d'articles. - -Tous les types de sources sont pris en charge. Lors de la mise à jour, les données (à partir d'un fichier ou d'une table) sont lues dans leur intégralité. - -Exemple de Configuration: - -``` xml - - - -``` - -ou - -``` sql -LAYOUT(HASHED()) -``` - -### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} - -Semblable à `hashed`, mais utilise moins de mémoire en faveur de plus D'utilisation du processeur. - -Exemple de Configuration: - -``` xml - - - -``` - -``` sql -LAYOUT(SPARSE_HASHED()) -``` - -### complex_key_hashed {#complex-key-hashed} - -Ce type de stockage est pour une utilisation avec composite [touches](external-dicts-dict-structure.md). Semblable à `hashed`. - -Exemple de Configuration: - -``` xml - - - -``` - -``` sql -LAYOUT(COMPLEX_KEY_HASHED()) -``` - -### range_hashed {#range-hashed} - -Le dictionnaire est stocké en mémoire sous la forme d'une table de hachage avec un tableau ordonné de gammes et leurs valeurs correspondantes. - -Cette méthode de stockage fonctionne de la même manière que hachée et permet d'utiliser des plages de date / heure (Type numérique arbitraire) en plus de la clé. - -Exemple: Le tableau contient des réductions pour chaque annonceur dans le format: - -``` text -+---------|-------------|-------------|------+ -| advertiser id | discount start date | discount end date | amount | -+===============+=====================+===================+========+ -| 123 | 2015-01-01 | 2015-01-15 | 0.15 | -+---------|-------------|-------------|------+ -| 123 | 2015-01-16 | 2015-01-31 | 0.25 | -+---------|-------------|-------------|------+ -| 456 | 2015-01-01 | 2015-01-15 | 0.05 | -+---------|-------------|-------------|------+ -``` - -Pour utiliser un échantillon pour les plages de dates, définissez `range_min` et `range_max` éléments dans le [structure](external-dicts-dict-structure.md). Ces éléments doivent contenir des éléments `name` et`type` (si `type` n'est pas spécifié, le type par défaut sera utilisé-Date). `type` peut être n'importe quel type numérique (Date / DateTime / UInt64 / Int32 / autres). - -Exemple: - -``` xml - - - Id - - - first - Date - - - last - Date - - ... -``` - -ou - -``` sql -CREATE DICTIONARY somedict ( - id UInt64, - first Date, - last Date -) -PRIMARY KEY id -LAYOUT(RANGE_HASHED()) -RANGE(MIN first MAX last) -``` - -Pour travailler avec ces dictionnaires, vous devez passer un argument supplémentaire à l' `dictGetT` fonction, pour laquelle une plage est sélectionnée: - -``` sql -dictGetT('dict_name', 'attr_name', id, date) -``` - -Cette fonction retourne la valeur pour l' `id`s et la plage de dates qui inclut la date passée. - -Détails de l'algorithme: - -- Si l' `id` est introuvable ou une plage n'est pas trouvé pour l' `id` il retourne la valeur par défaut pour le dictionnaire. -- S'il y a des plages qui se chevauchent, vous pouvez en utiliser. -- Si le délimiteur est `NULL` ou une date non valide (telle que 1900-01-01 ou 2039-01-01), la plage est laissée ouverte. La gamme peut être ouverte des deux côtés. - -Exemple de Configuration: - -``` xml - - - - ... - - - - - - - - Abcdef - - - StartTimeStamp - UInt64 - - - EndTimeStamp - UInt64 - - - XXXType - String - - - - - - -``` - -ou - -``` sql -CREATE DICTIONARY somedict( - Abcdef UInt64, - StartTimeStamp UInt64, - EndTimeStamp UInt64, - XXXType String DEFAULT '' -) -PRIMARY KEY Abcdef -RANGE(MIN StartTimeStamp MAX EndTimeStamp) -``` - -### cache {#cache} - -Le dictionnaire est stocké dans un cache qui a un nombre fixe de cellules. Ces cellules contiennent des éléments fréquemment utilisés. - -Lors de la recherche d'un dictionnaire, le cache est recherché en premier. Pour chaque bloc de données, toutes les clés qui ne sont pas trouvées dans le cache ou qui sont obsolètes sont demandées à la source en utilisant `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. Les données reçues sont ensuite écrites dans le cache. - -Pour les dictionnaires de cache, l'expiration [vie](external-dicts-dict-lifetime.md) des données dans le cache peuvent être définies. Si plus de temps que `lifetime` passé depuis le chargement des données dans une cellule, la valeur de la cellule n'est pas utilisée et elle est demandée à nouveau la prochaine fois qu'elle doit être utilisée. -C'est la moins efficace de toutes les façons de stocker les dictionnaires. La vitesse du cache dépend fortement des paramètres corrects et que le scénario d'utilisation. Un dictionnaire de type de cache fonctionne bien uniquement lorsque les taux de réussite sont suffisamment élevés (recommandé 99% et plus). Vous pouvez afficher le taux de réussite moyen dans le `system.dictionaries` table. - -Pour améliorer les performances du cache, utilisez une sous-requête avec `LIMIT`, et appelez la fonction avec le dictionnaire en externe. - -Soutenu [source](external-dicts-dict-sources.md): MySQL, ClickHouse, exécutable, HTTP. - -Exemple de paramètres: - -``` xml - - - - 1000000000 - - -``` - -ou - -``` sql -LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) -``` - -Définissez une taille de cache suffisamment grande. Vous devez expérimenter pour sélectionner le nombre de cellules: - -1. Définissez une valeur. -2. Exécutez les requêtes jusqu'à ce que le cache soit complètement plein. -3. Évaluer la consommation de mémoire en utilisant le `system.dictionaries` table. -4. Augmentez ou diminuez le nombre de cellules jusqu'à ce que la consommation de mémoire requise soit atteinte. - -!!! warning "Avertissement" - N'utilisez pas ClickHouse comme source, car le traitement des requêtes avec des lectures aléatoires est lent. - -### complex_key_cache {#complex-key-cache} - -Ce type de stockage est pour une utilisation avec composite [touches](external-dicts-dict-structure.md). Semblable à `cache`. - -### direct {#direct} - -Le dictionnaire n'est pas stocké dans la mémoire et va directement à la source, pendant le traitement d'une demande. - -La clé du dictionnaire a le `UInt64` type. - -Tous les types de [source](external-dicts-dict-sources.md), sauf les fichiers locaux, sont pris en charge. - -Exemple de Configuration: - -``` xml - - - -``` - -ou - -``` sql -LAYOUT(DIRECT()) -``` - -### complex_key_direct {#complex-key-direct} - -Ce type de stockage est destiné à être utilisé avec des [clés](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md) composites. Similaire à `direct` - -### ip_trie {#ip-trie} - -Ce type de stockage permet de mapper des préfixes de réseau (adresses IP) à des métadonnées telles que ASN. - -Exemple: la table contient les préfixes de réseau et leur correspondant en tant que numéro et Code de pays: - -``` text - +-----------|-----|------+ - | prefix | asn | cca2 | - +=================+=======+========+ - | 202.79.32.0/20 | 17501 | NP | - +-----------|-----|------+ - | 2620:0:870::/48 | 3856 | US | - +-----------|-----|------+ - | 2a02:6b8:1::/48 | 13238 | RU | - +-----------|-----|------+ - | 2001:db8::/32 | 65536 | ZZ | - +-----------|-----|------+ -``` - -Lorsque vous utilisez ce type de mise en page, la structure doit avoir une clé composite. - -Exemple: - -``` xml - - - - prefix - String - - - - asn - UInt32 - - - - cca2 - String - ?? - - ... - - - - true - - -``` - -ou - -``` sql -CREATE DICTIONARY somedict ( - prefix String, - asn UInt32, - cca2 String DEFAULT '??' -) -PRIMARY KEY prefix -``` - -La clé ne doit avoir qu'un seul attribut de type chaîne contenant un préfixe IP autorisé. Les autres types ne sont pas encore pris en charge. - -Pour les requêtes, vous devez utiliser les mêmes fonctions (`dictGetT` avec un n-uplet) comme pour les dictionnaires avec des clés composites: - -``` sql -dictGetT('dict_name', 'attr_name', tuple(ip)) -``` - -La fonction prend soit `UInt32` pour IPv4, ou `FixedString(16)` pour IPv6: - -``` sql -dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) -``` - -Les autres types ne sont pas encore pris en charge. La fonction renvoie l'attribut du préfixe correspondant à cette adresse IP. S'il y a chevauchement des préfixes, le plus spécifique est retourné. - -Les données doit complètement s'intégrer dans la RAM. - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_layout/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md deleted file mode 100644 index 8ce78919ff1..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ /dev/null @@ -1,91 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 42 -toc_title: "Mises \xC0 Jour Du Dictionnaire" ---- - -# Mises À Jour Du Dictionnaire {#dictionary-updates} - -ClickHouse met périodiquement à jour les dictionnaires. L'intervalle de mise à jour pour les dictionnaires entièrement téléchargés et l'intervalle d'invalidation pour les dictionnaires `` tag en quelques secondes. - -Les mises à jour du dictionnaire (autres que le chargement pour la première utilisation) ne bloquent pas les requêtes. Lors des mises à jour, l'ancienne version d'un dictionnaire est utilisée. Si une erreur se produit pendant une mise à jour, l'erreur est écrite dans le journal du serveur et les requêtes continuent d'utiliser l'ancienne version des dictionnaires. - -Exemple de paramètres: - -``` xml - - ... - 300 - ... - -``` - -``` sql -CREATE DICTIONARY (...) -... -LIFETIME(300) -... -``` - -Paramètre `0` (`LIFETIME(0)`) empêche la mise à jour des dictionnaires. - -Vous pouvez définir un intervalle de temps pour les mises à niveau, et ClickHouse choisira un temps uniformément aléatoire dans cette plage. Ceci est nécessaire pour répartir la charge sur la source du dictionnaire lors de la mise à niveau sur un grand nombre de serveurs. - -Exemple de paramètres: - -``` xml - - ... - - 300 - 360 - - ... - -``` - -ou - -``` sql -LIFETIME(MIN 300 MAX 360) -``` - -Si `0` et `0`, ClickHouse ne recharge pas le dictionnaire par timeout. -Dans ce cas, ClickHouse peut recharger le dictionnaire plus tôt si le fichier de configuration du dictionnaire a été `SYSTEM RELOAD DICTIONARY` la commande a été exécutée. - -Lors de la mise à niveau des dictionnaires, le serveur ClickHouse applique une logique différente selon le type de [source](external-dicts-dict-sources.md): - -Lors de la mise à niveau des dictionnaires, le serveur ClickHouse applique une logique différente selon le type de [source](external-dicts-dict-sources.md): - -- Pour un fichier texte, il vérifie l'heure de la modification. Si l'heure diffère de l'heure enregistrée précédemment, le dictionnaire est mis à jour. -- Pour les tables MyISAM, l'Heure de modification est vérifiée à l'aide d'un `SHOW TABLE STATUS` requête. -- Les dictionnaires d'autres sources sont mis à jour à chaque fois par défaut. - -Pour les sources MySQL (InnoDB), ODBC et ClickHouse, vous pouvez configurer une requête qui mettra à jour les dictionnaires uniquement s'ils ont vraiment changé, plutôt que chaque fois. Pour ce faire, suivez ces étapes: - -- La table de dictionnaire doit avoir un champ qui change toujours lorsque les données source sont mises à jour. -- Les paramètres de la source doivent spécifier une requête qui récupère le champ de modification. Le serveur ClickHouse interprète le résultat de la requête comme une ligne, et si cette ligne a changé par rapport à son état précédent, le dictionnaire est mis à jour. Spécifier la requête dans le `` champ dans les paramètres pour le [source](external-dicts-dict-sources.md). - -Exemple de paramètres: - -``` xml - - ... - - ... - SELECT update_time FROM dictionary_source where id = 1 - - ... - -``` - -ou - -``` sql -... -SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source where id = 1')) -... -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_lifetime/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md deleted file mode 100644 index 4c608fa7188..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ /dev/null @@ -1,630 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 43 -toc_title: Sources de dictionnaires externes ---- - -# Sources de dictionnaires externes {#dicts-external-dicts-dict-sources} - -Externe dictionnaire peut être connecté à partir de nombreuses sources différentes. - -Si dictionary est configuré à l'aide de xml-file, la configuration ressemble à ceci: - -``` xml - - - ... - - - - - - ... - - ... - -``` - -En cas de [DDL-requête](../../statements/create.md#create-dictionary-query), configuration égale ressemblera à: - -``` sql -CREATE DICTIONARY dict_name (...) -... -SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration -... -``` - -La source est configurée dans le `source` section. - -Pour les types de source [Fichier Local](#dicts-external_dicts_dict_sources-local_file), [Fichier exécutable](#dicts-external_dicts_dict_sources-executable), [HTTP(S)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) -les paramètres optionnels sont disponibles: - -``` xml - - - /opt/dictionaries/os.tsv - TabSeparated - - - 0 - - -``` - -ou - -``` sql -SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) -SETTINGS(format_csv_allow_single_quotes = 0) -``` - -Les Types de sources (`source_type`): - -- [Fichier Local](#dicts-external_dicts_dict_sources-local_file) -- [Fichier exécutable](#dicts-external_dicts_dict_sources-executable) -- [HTTP(S)](#dicts-external_dicts_dict_sources-http) -- DBMS - - [ODBC](#dicts-external_dicts_dict_sources-odbc) - - [MySQL](#dicts-external_dicts_dict_sources-mysql) - - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) - - [Redis](#dicts-external_dicts_dict_sources-redis) - -## Fichier Local {#dicts-external_dicts_dict_sources-local_file} - -Exemple de paramètres: - -``` xml - - - /opt/dictionaries/os.tsv - TabSeparated - - -``` - -ou - -``` sql -SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) -``` - -Définition des champs: - -- `path` – The absolute path to the file. -- `format` – The file format. All the formats described in “[Format](../../../interfaces/formats.md#formats)” sont pris en charge. - -## Fichier Exécutable {#dicts-external_dicts_dict_sources-executable} - -Travailler avec des fichiers exécutables en dépend [comment le dictionnaire est stocké dans la mémoire](external-dicts-dict-layout.md). Si le dictionnaire est stocké en utilisant `cache` et `complex_key_cache`, Clickhouse demande les clés nécessaires en envoyant une requête au STDIN du fichier exécutable. Sinon, ClickHouse démarre le fichier exécutable et traite sa sortie comme des données de dictionnaire. - -Exemple de paramètres: - -``` xml - - - cat /opt/dictionaries/os.tsv - TabSeparated - - -``` - -ou - -``` sql -SOURCE(EXECUTABLE(command 'cat /opt/dictionaries/os.tsv' format 'TabSeparated')) -``` - -Définition des champs: - -- `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). -- `format` – The file format. All the formats described in “[Format](../../../interfaces/formats.md#formats)” sont pris en charge. - -## Http(s) {#dicts-external_dicts_dict_sources-http} - -Travailler avec un serveur HTTP (S) dépend de [comment le dictionnaire est stocké dans la mémoire](external-dicts-dict-layout.md). Si le dictionnaire est stocké en utilisant `cache` et `complex_key_cache`, Clickhouse demande les clés nécessaires en envoyant une demande via le `POST` méthode. - -Exemple de paramètres: - -``` xml - - - http://[::1]/os.tsv - TabSeparated - - user - password - - -
- API-KEY - key -
-
-
- -``` - -ou - -``` sql -SOURCE(HTTP( - url 'http://[::1]/os.tsv' - format 'TabSeparated' - credentials(user 'user' password 'password') - headers(header(name 'API-KEY' value 'key')) -)) -``` - -Pour que ClickHouse accède à une ressource HTTPS, vous devez [configurer openSSL](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl) dans la configuration du serveur. - -Définition des champs: - -- `url` – The source URL. -- `format` – The file format. All the formats described in “[Format](../../../interfaces/formats.md#formats)” sont pris en charge. -- `credentials` – Basic HTTP authentication. Optional parameter. - - `user` – Username required for the authentication. - - `password` – Password required for the authentication. -- `headers` – All custom HTTP headers entries used for the HTTP request. Optional parameter. - - `header` – Single HTTP header entry. - - `name` – Identifiant name used for the header send on the request. - - `value` – Value set for a specific identifiant name. - -## ODBC {#dicts-external_dicts_dict_sources-odbc} - -Vous pouvez utiliser cette méthode pour connecter n'importe quelle base de données dotée d'un pilote ODBC. - -Exemple de paramètres: - -``` xml - - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY -
- -``` - -ou - -``` sql -SOURCE(ODBC( - db 'DatabaseName' - table 'SchemaName.TableName' - connection_string 'DSN=some_parameters' - invalidate_query 'SQL_QUERY' -)) -``` - -Définition des champs: - -- `db` – Name of the database. Omit it if the database name is set in the `` paramètre. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Mise à jour des dictionnaires](external-dicts-dict-lifetime.md). - -ClickHouse reçoit des symboles de citation D'ODBC-driver et cite tous les paramètres des requêtes au pilote, il est donc nécessaire de définir le nom de la table en conséquence sur le cas du nom de la table dans la base de données. - -Si vous avez des problèmes avec des encodages lors de l'utilisation d'Oracle, consultez le [FAQ](../../../faq/general.md#oracle-odbc-encodings) article. - -### Vulnérabilité connue de la fonctionnalité du dictionnaire ODBC {#known-vulnerability-of-the-odbc-dictionary-functionality} - -!!! attention "Attention" - Lors de la connexion à la base de données via le paramètre de connexion du pilote ODBC `Servername` peut être substitué. Dans ce cas, les valeurs de `USERNAME` et `PASSWORD` de `odbc.ini` sont envoyés au serveur distant et peuvent être compromis. - -**Exemple d'utilisation non sécurisée** - -Configurons unixODBC pour PostgreSQL. Le contenu de `/etc/odbc.ini`: - -``` text -[gregtest] -Driver = /usr/lib/psqlodbca.so -Servername = localhost -PORT = 5432 -DATABASE = test_db -#OPTION = 3 -USERNAME = test -PASSWORD = test -``` - -Si vous faites alors une requête telle que - -``` sql -SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); -``` - -Le pilote ODBC enverra des valeurs de `USERNAME` et `PASSWORD` de `odbc.ini` de `some-server.com`. - -### Exemple de connexion Postgresql {#example-of-connecting-postgresql} - -Ubuntu OS. - -Installation d'unixODBC et du pilote ODBC pour PostgreSQL: - -``` bash -$ sudo apt-get install -y unixodbc odbcinst odbc-postgresql -``` - -Configuration `/etc/odbc.ini` (ou `~/.odbc.ini`): - -``` text - [DEFAULT] - Driver = myconnection - - [myconnection] - Description = PostgreSQL connection to my_db - Driver = PostgreSQL Unicode - Database = my_db - Servername = 127.0.0.1 - UserName = username - Password = password - Port = 5432 - Protocol = 9.3 - ReadOnly = No - RowVersioning = No - ShowSystemTables = No - ConnSettings = -``` - -La configuration du dictionnaire dans ClickHouse: - -``` xml - - - table_name - - - - - DSN=myconnection - postgresql_table
-
- - - 300 - 360 - - - - - - - id - - - some_column - UInt64 - 0 - - -
-
-``` - -ou - -``` sql -CREATE DICTIONARY table_name ( - id UInt64, - some_column UInt64 DEFAULT 0 -) -PRIMARY KEY id -SOURCE(ODBC(connection_string 'DSN=myconnection' table 'postgresql_table')) -LAYOUT(HASHED()) -LIFETIME(MIN 300 MAX 360) -``` - -Vous devrez peut-être modifier `odbc.ini` pour spécifier le chemin d'accès complet à la bibliothèque avec le conducteur `DRIVER=/usr/local/lib/psqlodbcw.so`. - -### Exemple de connexion à MS SQL Server {#example-of-connecting-ms-sql-server} - -Ubuntu OS. - -Installation du pilote: : - -``` bash -$ sudo apt-get install tdsodbc freetds-bin sqsh -``` - -Configuration du pilote: - -``` bash - $ cat /etc/freetds/freetds.conf - ... - - [MSSQL] - host = 192.168.56.101 - port = 1433 - tds version = 7.0 - client charset = UTF-8 - - $ cat /etc/odbcinst.ini - ... - - [FreeTDS] - Description = FreeTDS - Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so - Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so - FileUsage = 1 - UsageCount = 5 - - $ cat ~/.odbc.ini - ... - - [MSSQL] - Description = FreeTDS - Driver = FreeTDS - Servername = MSSQL - Database = test - UID = test - PWD = test - Port = 1433 -``` - -Configuration du dictionnaire dans ClickHouse: - -``` xml - - - test - - - dict
- DSN=MSSQL;UID=test;PWD=test -
- - - - 300 - 360 - - - - - - - - - k - - - s - String - - - -
-
-``` - -ou - -``` sql -CREATE DICTIONARY test ( - k UInt64, - s String DEFAULT '' -) -PRIMARY KEY k -SOURCE(ODBC(table 'dict' connection_string 'DSN=MSSQL;UID=test;PWD=test')) -LAYOUT(FLAT()) -LIFETIME(MIN 300 MAX 360) -``` - -## DBMS {#dbms} - -### Mysql {#dicts-external_dicts_dict_sources-mysql} - -Exemple de paramètres: - -``` xml - - - 3306 - clickhouse - qwerty - - example01-1 - 1 - - - example01-2 - 1 - - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - -ou - -``` sql -SOURCE(MYSQL( - port 3306 - user 'clickhouse' - password 'qwerty' - replica(host 'example01-1' priority 1) - replica(host 'example01-2' priority 1) - db 'db_name' - table 'table_name' - where 'id=10' - invalidate_query 'SQL_QUERY' -)) -``` - -Définition des champs: - -- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). - -- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `replica` – Section of replica configurations. There can be multiple sections. - - - `replica/host` – The MySQL host. - - `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. - -- `db` – Name of the database. - -- `table` – Name of the table. - -- `where` – The selection criteria. The syntax for conditions is the same as for `WHERE` clause dans MySQL, par exemple, `id > 10 AND id < 20`. Paramètre facultatif. - -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Mise à jour des dictionnaires](external-dicts-dict-lifetime.md). - -MySQL peut être connecté sur un hôte local via des sockets. Pour ce faire, définissez `host` et `socket`. - -Exemple de paramètres: - -``` xml - - - localhost - /path/to/socket/file.sock - clickhouse - qwerty - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - -ou - -``` sql -SOURCE(MYSQL( - host 'localhost' - socket '/path/to/socket/file.sock' - user 'clickhouse' - password 'qwerty' - db 'db_name' - table 'table_name' - where 'id=10' - invalidate_query 'SQL_QUERY' -)) -``` - -### ClickHouse {#dicts-external_dicts_dict_sources-clickhouse} - -Exemple de paramètres: - -``` xml - - - example01-01-1 - 9000 - default - - default - ids
- id=10 -
- -``` - -ou - -``` sql -SOURCE(CLICKHOUSE( - host 'example01-01-1' - port 9000 - user 'default' - password '' - db 'default' - table 'ids' - where 'id=10' -)) -``` - -Définition des champs: - -- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distribué](../../../engines/table-engines/special/distributed.md) table et entrez-le dans les configurations suivantes. -- `port` – The port on the ClickHouse server. -- `user` – Name of the ClickHouse user. -- `password` – Password of the ClickHouse user. -- `db` – Name of the database. -- `table` – Name of the table. -- `where` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Mise à jour des dictionnaires](external-dicts-dict-lifetime.md). - -### Mongodb {#dicts-external_dicts_dict_sources-mongodb} - -Exemple de paramètres: - -``` xml - - - localhost - 27017 - - - test - dictionary_source - - -``` - -ou - -``` sql -SOURCE(MONGO( - host 'localhost' - port 27017 - user '' - password '' - db 'test' - collection 'dictionary_source' -)) -``` - -Définition des champs: - -- `host` – The MongoDB host. -- `port` – The port on the MongoDB server. -- `user` – Name of the MongoDB user. -- `password` – Password of the MongoDB user. -- `db` – Name of the database. -- `collection` – Name of the collection. - -### Redis {#dicts-external_dicts_dict_sources-redis} - -Exemple de paramètres: - -``` xml - - - localhost - 6379 - simple - 0 - - -``` - -ou - -``` sql -SOURCE(REDIS( - host 'localhost' - port 6379 - storage_type 'simple' - db_index 0 -)) -``` - -Définition des champs: - -- `host` – The Redis host. -- `port` – The port on the Redis server. -- `storage_type` – The structure of internal Redis storage using for work with keys. `simple` est pour les sources simples et pour les sources à clé unique hachées, `hash_map` est pour les sources hachées avec deux clés. Les sources À Distance et les sources de cache à clé complexe ne sont pas prises en charge. Peut être omis, la valeur par défaut est `simple`. -- `db_index` – The specific numeric index of Redis logical database. May be omitted, default value is 0. - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_sources/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md deleted file mode 100644 index 1b9215baf06..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: "Cl\xE9 et champs du dictionnaire" ---- - -# Clé et champs du dictionnaire {#dictionary-key-and-fields} - -Le `` la clause décrit la clé du dictionnaire et les champs disponibles pour les requêtes. - -Description XML: - -``` xml - - - - Id - - - - - - - ... - - - -``` - -Les attributs sont décrits dans les éléments: - -- `` — [La colonne de la clé](external-dicts-dict-structure.md#ext_dict_structure-key). -- `` — [Colonne de données](external-dicts-dict-structure.md#ext_dict_structure-attributes). Il peut y avoir un certain nombre d'attributs. - -Requête DDL: - -``` sql -CREATE DICTIONARY dict_name ( - Id UInt64, - -- attributes -) -PRIMARY KEY Id -... -``` - -Les attributs sont décrits dans le corps de la requête: - -- `PRIMARY KEY` — [La colonne de la clé](external-dicts-dict-structure.md#ext_dict_structure-key) -- `AttrName AttrType` — [Colonne de données](external-dicts-dict-structure.md#ext_dict_structure-attributes). Il peut y avoir un certain nombre d'attributs. - -## Clé {#ext_dict_structure-key} - -ClickHouse prend en charge les types de clés suivants: - -- Touche numérique. `UInt64`. Défini dans le `` tag ou en utilisant `PRIMARY KEY` mot. -- Clé Composite. Ensemble de valeurs de types différents. Défini dans la balise `` ou `PRIMARY KEY` mot. - -Une structure xml peut contenir `` ou ``. DDL-requête doit contenir unique `PRIMARY KEY`. - -!!! warning "Avertissement" - Vous ne devez pas décrire clé comme un attribut. - -### Touche Numérique {#ext_dict-numeric-key} - -Type: `UInt64`. - -Exemple de Configuration: - -``` xml - - Id - -``` - -Champs de Configuration: - -- `name` – The name of the column with keys. - -Pour DDL-requête: - -``` sql -CREATE DICTIONARY ( - Id UInt64, - ... -) -PRIMARY KEY Id -... -``` - -- `PRIMARY KEY` – The name of the column with keys. - -### Clé Composite {#composite-key} - -La clé peut être un `tuple` de tous les types de champs. Le [disposition](external-dicts-dict-layout.md) dans ce cas, doit être `complex_key_hashed` ou `complex_key_cache`. - -!!! tip "Conseil" - Une clé composite peut être constitué d'un seul élément. Cela permet d'utiliser une chaîne comme clé, par exemple. - -La structure de clé est définie dans l'élément ``. Les principaux champs sont spécifiés dans le même format que le dictionnaire [attribut](external-dicts-dict-structure.md). Exemple: - -``` xml - - - - field1 - String - - - field2 - UInt32 - - ... - -... -``` - -ou - -``` sql -CREATE DICTIONARY ( - field1 String, - field2 String - ... -) -PRIMARY KEY field1, field2 -... -``` - -Pour une requête à l' `dictGet*` fonction, un tuple est passé comme clé. Exemple: `dictGetString('dict_name', 'attr_name', tuple('string for field1', num_for_field2))`. - -## Attribut {#ext_dict_structure-attributes} - -Exemple de Configuration: - -``` xml - - ... - - Name - ClickHouseDataType - - rand64() - true - true - true - - -``` - -ou - -``` sql -CREATE DICTIONARY somename ( - Name ClickHouseDataType DEFAULT '' EXPRESSION rand64() HIERARCHICAL INJECTIVE IS_OBJECT_ID -) -``` - -Champs de Configuration: - -| Balise | Description | Requis | -|------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------| -| `name` | Nom de la colonne. | Oui | -| `type` | Type de données ClickHouse.
ClickHouse tente de convertir la valeur du dictionnaire vers le type de données spécifié. Par exemple, pour MySQL, le champ peut être `TEXT`, `VARCHAR`, ou `BLOB` dans la table source MySQL, mais il peut être téléchargé comme `String` à ClickHouse.
[Nullable](../../../sql-reference/data-types/nullable.md) n'est pas pris en charge. | Oui | -| `null_value` | Valeur par défaut pour un élément inexistant.
Dans l'exemple, c'est une chaîne vide. Vous ne pouvez pas utiliser `NULL` dans ce domaine. | Oui | -| `expression` | [Expression](../../syntax.md#syntax-expressions) que ClickHouse s'exécute sur la valeur.
L'expression peut être un nom de colonne dans la base de données SQL distante. Ainsi, vous pouvez l'utiliser pour créer un alias pour la colonne à distance.

Valeur par défaut: aucune expression. | Aucun | -| `hierarchical` | Si `true`, l'attribut contient la valeur d'un parent clé de la clé actuelle. Voir [Dictionnaires Hiérarchiques](external-dicts-dict-hierarchical.md).

Valeur par défaut: `false`. | Aucun | -| `injective` | Indicateur qui indique si le `id -> attribute` l'image est [injective](https://en.wikipedia.org/wiki/Injective_function).
Si `true`, ClickHouse peut automatiquement placer après le `GROUP BY` clause les requêtes aux dictionnaires avec injection. Habituellement, il réduit considérablement le montant de ces demandes.

Valeur par défaut: `false`. | Aucun | -| `is_object_id` | Indicateur qui indique si la requête est exécutée pour un document MongoDB par `ObjectID`.

Valeur par défaut: `false`. | Aucun | - -## Voir Aussi {#see-also} - -- [Fonctions pour travailler avec des dictionnaires externes](../../../sql-reference/functions/ext-dict-functions.md). - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict_structure/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md deleted file mode 100644 index 3bb8884df2f..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ /dev/null @@ -1,53 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: Configuration D'un dictionnaire externe ---- - -# Configuration D'un dictionnaire externe {#dicts-external-dicts-dict} - -Si dictionary est configuré à l'aide d'un fichier xml, than dictionary configuration a la structure suivante: - -``` xml - - dict_name - - - - - - - - - - - - - - - - - -``` - -Correspondant [DDL-requête](../../statements/create.md#create-dictionary-query) a la structure suivante: - -``` sql -CREATE DICTIONARY dict_name -( - ... -- attributes -) -PRIMARY KEY ... -- complex or single key configuration -SOURCE(...) -- Source configuration -LAYOUT(...) -- Memory layout configuration -LIFETIME(...) -- Lifetime of dictionary in memory -``` - -- `name` – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. -- [source](external-dicts-dict-sources.md) — Source of the dictionary. -- [disposition](external-dicts-dict-layout.md) — Dictionary layout in memory. -- [structure](external-dicts-dict-structure.md) — Structure of the dictionary . A key and attributes that can be retrieved by this key. -- [vie](external-dicts-dict-lifetime.md) — Frequency of dictionary updates. - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts_dict/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md deleted file mode 100644 index d68b7a7f112..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ /dev/null @@ -1,62 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: "Description G\xE9n\xE9rale" ---- - -# Dictionnaires Externes {#dicts-external-dicts} - -Vous pouvez ajouter vos propres dictionnaires à partir de diverses sources de données. La source de données d'un dictionnaire peut être un texte local ou un fichier exécutable, une ressource HTTP(S) ou un autre SGBD. Pour plus d'informations, voir “[Sources pour les dictionnaires externes](external-dicts-dict-sources.md)”. - -ClickHouse: - -- Stocke entièrement ou partiellement les dictionnaires en RAM. -- Met à jour périodiquement les dictionnaires et charge dynamiquement les valeurs manquantes. En d'autres mots, les dictionnaires peuvent être chargés dynamiquement. -- Permet de créer des dictionnaires externes avec des fichiers xml ou [Les requêtes DDL](../../statements/create.md#create-dictionary-query). - -La configuration des dictionnaires externes peut être située dans un ou plusieurs fichiers xml. Le chemin d'accès à la configuration spécifiée dans le [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) paramètre. - -Les dictionnaires peuvent être chargés au démarrage du serveur ou à la première utilisation, en fonction [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) paramètre. - -Le [dictionnaire](../../../operations/system-tables.md#system_tables-dictionaries) la table système contient des informations sur les dictionnaires configurés sur le serveur. Pour chaque dictionnaire, vous pouvez y trouver: - -- Statut du dictionnaire. -- Paramètres de Configuration. -- Des métriques telles que la quantité de RAM allouée pour le dictionnaire ou un certain nombre de requêtes depuis que le dictionnaire a été chargé avec succès. - -Le fichier de configuration du dictionnaire a le format suivant: - -``` xml - - An optional element with any content. Ignored by the ClickHouse server. - - - /etc/metrika.xml - - - - - - - - -``` - -Vous pouvez [configurer](external-dicts-dict.md) le nombre de dictionnaires dans le même fichier. - -[Requêtes DDL pour les dictionnaires](../../statements/create.md#create-dictionary-query) ne nécessite aucun enregistrement supplémentaire dans la configuration du serveur. Ils permettent de travailler avec des dictionnaires en tant qu'entités de première classe, comme des tables ou des vues. - -!!! attention "Attention" - Vous pouvez convertir les valeurs pour un petit dictionnaire en le décrivant dans un `SELECT` requête (voir la [transformer](../../../sql-reference/functions/other-functions.md) fonction). Cette fonctionnalité n'est pas liée aux dictionnaires externes. - -## Voir Aussi {#ext-dicts-see-also} - -- [Configuration D'un dictionnaire externe](external-dicts-dict.md) -- [Stockage des dictionnaires en mémoire](external-dicts-dict-layout.md) -- [Mises À Jour Du Dictionnaire](external-dicts-dict-lifetime.md) -- [Sources de dictionnaires externes](external-dicts-dict-sources.md) -- [Clé et champs du dictionnaire](external-dicts-dict-structure.md) -- [Fonctions pour travailler avec des dictionnaires externes](../../../sql-reference/functions/ext-dict-functions.md) - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts/) diff --git a/docs/fr/sql-reference/dictionaries/external-dictionaries/index.md b/docs/fr/sql-reference/dictionaries/external-dictionaries/index.md deleted file mode 100644 index 109220205dd..00000000000 --- a/docs/fr/sql-reference/dictionaries/external-dictionaries/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Dictionnaires Externes -toc_priority: 37 ---- - - diff --git a/docs/fr/sql-reference/dictionaries/index.md b/docs/fr/sql-reference/dictionaries/index.md deleted file mode 100644 index 3ec31085cc5..00000000000 --- a/docs/fr/sql-reference/dictionaries/index.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Dictionnaire -toc_priority: 35 -toc_title: Introduction ---- - -# Dictionnaire {#dictionaries} - -Un dictionnaire est une cartographie (`key -> attributes`) qui est pratique pour différents types de listes de référence. - -ClickHouse prend en charge des fonctions spéciales pour travailler avec des dictionnaires qui peuvent être utilisés dans les requêtes. Il est plus facile et plus efficace d'utiliser des dictionnaires avec des fonctions que par une `JOIN` avec des tableaux de référence. - -[NULL](../../sql-reference/syntax.md#null-literal) les valeurs ne peuvent pas être stockées dans un dictionnaire. - -Supports ClickHouse: - -- [Construit-dans les dictionnaires](internal-dicts.md#internal_dicts) avec un [ensemble de fonctions](../../sql-reference/functions/ym-dict-functions.md). -- [Plug-in (externe) dictionnaires](external-dictionaries/external-dicts.md#dicts-external-dicts) avec un [ensemble de fonctions](../../sql-reference/functions/ext-dict-functions.md). - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/) diff --git a/docs/fr/sql-reference/dictionaries/internal-dicts.md b/docs/fr/sql-reference/dictionaries/internal-dicts.md deleted file mode 100644 index 607936031a1..00000000000 --- a/docs/fr/sql-reference/dictionaries/internal-dicts.md +++ /dev/null @@ -1,55 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: Dictionnaires Internes ---- - -# Dictionnaires Internes {#internal_dicts} - -ClickHouse contient une fonction intégrée pour travailler avec une géobase. - -Cela vous permet de: - -- Utilisez L'ID d'une région pour obtenir son nom dans la langue souhaitée. -- Utilisez L'ID d'une région pour obtenir L'ID d'une ville, d'une région, d'un district fédéral, d'un pays ou d'un continent. -- Vérifiez si une région fait partie d'une autre région. -- Obtenez une chaîne de régions parentes. - -Toutes les fonctions prennent en charge “translocality,” la capacité d'utiliser simultanément différentes perspectives sur la propriété de la région. Pour plus d'informations, consultez la section “Functions for working with Yandex.Metrica dictionaries”. - -Les dictionnaires internes sont désactivés dans le package par défaut. -Pour les activer, décommentez les paramètres `path_to_regions_hierarchy_file` et `path_to_regions_names_files` dans le fichier de configuration du serveur. - -La géobase est chargée à partir de fichiers texte. - -Place de la `regions_hierarchy*.txt` les fichiers dans le `path_to_regions_hierarchy_file` répertoire. Ce paramètre de configuration doit contenir le chemin `regions_hierarchy.txt` fichier (la hiérarchie régionale par défaut), et les autres fichiers (`regions_hierarchy_ua.txt`) doit être situé dans le même répertoire. - -Mettre le `regions_names_*.txt` les fichiers dans le `path_to_regions_names_files` répertoire. - -Vous pouvez également créer ces fichiers vous-même. Le format de fichier est le suivant: - -`regions_hierarchy*.txt`: TabSeparated (pas d'en-tête), colonnes: - -- région de l'ID (`UInt32`) -- ID de région parent (`UInt32`) -- type de région (`UInt8`): 1-continent, 3-pays, 4-district fédéral, 5-région, 6-ville; les autres types n'ont pas de valeurs -- population (`UInt32`) — optional column - -`regions_names_*.txt`: TabSeparated (pas d'en-tête), colonnes: - -- région de l'ID (`UInt32`) -- nom de la région (`String`) — Can't contain tabs or line feeds, even escaped ones. - -Un tableau plat est utilisé pour stocker dans la RAM. Pour cette raison, les ID ne devraient pas dépasser un million. - -Les dictionnaires peuvent être mis à jour sans redémarrer le serveur. Cependant, l'ensemble des dictionnaires n'est pas mis à jour. -Pour les mises à jour, les temps de modification du fichier sont vérifiés. Si un fichier a été modifié, le dictionnaire est mis à jour. -L'intervalle de vérification des modifications est configuré dans le `builtin_dictionaries_reload_interval` paramètre. -Les mises à jour du dictionnaire (autres que le chargement lors de la première utilisation) ne bloquent pas les requêtes. Lors des mises à jour, les requêtes utilisent les anciennes versions des dictionnaires. Si une erreur se produit pendant une mise à jour, l'erreur est écrite dans le journal du serveur et les requêtes continuent d'utiliser l'ancienne version des dictionnaires. - -Nous vous recommandons de mettre à jour périodiquement les dictionnaires avec la géobase. Lors d'une mise à jour, générez de nouveaux fichiers et écrivez-les dans un emplacement séparé. Lorsque tout est prêt, renommez - les en fichiers utilisés par le serveur. - -Il existe également des fonctions pour travailler avec les identifiants du système d'exploitation et Yandex.Moteurs de recherche Metrica, mais ils ne devraient pas être utilisés. - -[Article Original](https://clickhouse.tech/docs/en/query_language/dicts/internal_dicts/) diff --git a/docs/fr/sql-reference/functions/arithmetic-functions.md b/docs/fr/sql-reference/functions/arithmetic-functions.md deleted file mode 100644 index c35fb104236..00000000000 --- a/docs/fr/sql-reference/functions/arithmetic-functions.md +++ /dev/null @@ -1,87 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 35 -toc_title: "Arithm\xE9tique" ---- - -# Fonctions Arithmétiques {#arithmetic-functions} - -Pour toutes les fonctions arithmétiques, le type de résultat est calculé comme le plus petit type de nombre dans lequel le résultat correspond, s'il existe un tel type. Le minimum est pris simultanément sur la base du nombre de bits, s'il est signé, et s'il flotte. S'il n'y a pas assez de bits, le type de bits le plus élevé est pris. - -Exemple: - -``` sql -SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 + 0 + 0) -``` - -``` text -┌─toTypeName(0)─┬─toTypeName(plus(0, 0))─┬─toTypeName(plus(plus(0, 0), 0))─┬─toTypeName(plus(plus(plus(0, 0), 0), 0))─┐ -│ UInt8 │ UInt16 │ UInt32 │ UInt64 │ -└───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ -``` - -Les fonctions arithmétiques fonctionnent pour n'importe quelle paire de types de UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32 ou Float64. - -Le débordement est produit de la même manière qu'en C++. - -## plus (A, B), opérateur a + b {#plusa-b-a-b-operator} - -Calcule la somme des nombres. -Vous pouvez également ajouter des nombres entiers avec une date ou la date et l'heure. Dans le cas d'une date, Ajouter un entier signifie ajouter le nombre de jours correspondant. Pour une date avec l'heure, cela signifie ajouter le nombre de secondes correspondant. - -## moins (A, B), opérateur a - b {#minusa-b-a-b-operator} - -Calcule la différence. Le résultat est toujours signé. - -You can also calculate integer numbers from a date or date with time. The idea is the same – see above for ‘plus’. - -## la multiplication(a, b), a \* et b \* de l'opérateur {#multiplya-b-a-b-operator} - -Calcule le produit des nombres. - -## diviser (A, B), opérateur a / b {#dividea-b-a-b-operator} - -Calcule le quotient des nombres. Le type de résultat est toujours un type à virgule flottante. -Il n'est pas de division entière. Pour la division entière, utilisez le ‘intDiv’ fonction. -En divisant par zéro vous obtenez ‘inf’, ‘-inf’, ou ‘nan’. - -## intDiv (a, b) {#intdiva-b} - -Calcule le quotient des nombres. Divise en entiers, arrondi vers le bas (par la valeur absolue). -Une exception est levée en divisant par zéro ou en divisant un nombre négatif minimal par moins un. - -## intDivOrZero(a, b) {#intdivorzeroa-b} - -Diffère de ‘intDiv’ en ce sens qu'il renvoie zéro en divisant par zéro ou en divisant un nombre négatif minimal par moins un. - -## opérateur modulo(A, B), A % B {#moduloa-b-a-b-operator} - -Calcule le reste après la division. -Si les arguments sont des nombres à virgule flottante, ils sont pré-convertis en entiers en supprimant la partie décimale. -Le reste est pris dans le même sens qu'en C++. La division tronquée est utilisée pour les nombres négatifs. -Une exception est levée en divisant par zéro ou en divisant un nombre négatif minimal par moins un. - -## moduloOrZero (a, b) {#moduloorzeroa-b} - -Diffère de ‘modulo’ en ce sens qu'il renvoie zéro lorsque le diviseur est nul. - -## annuler (a), - un opérateur {#negatea-a-operator} - -Calcule un nombre avec le signe inverse. Le résultat est toujours signé. - -## abs(un) {#arithm_func-abs} - -Calcule la valeur absolue d'un nombre (un). Autrement dit, si un \< 0, Il renvoie-A. pour les types non signés, il ne fait rien. Pour les types entiers signés, il renvoie un nombre non signé. - -## pgcd(a, b) {#gcda-b} - -Renvoie le plus grand diviseur commun des nombres. -Une exception est levée en divisant par zéro ou en divisant un nombre négatif minimal par moins un. - -## ppcm(a, b) {#lcma-b} - -Renvoie le multiple le moins commun des nombres. -Une exception est levée en divisant par zéro ou en divisant un nombre négatif minimal par moins un. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/arithmetic_functions/) diff --git a/docs/fr/sql-reference/functions/array-functions.md b/docs/fr/sql-reference/functions/array-functions.md deleted file mode 100644 index 40568841372..00000000000 --- a/docs/fr/sql-reference/functions/array-functions.md +++ /dev/null @@ -1,1061 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 46 -toc_title: Travailler avec des tableaux ---- - -# Fonctions pour travailler avec des tableaux {#functions-for-working-with-arrays} - -## vide {#function-empty} - -Retourne 1 pour un tableau vide, ou 0 pour un non-vide. -Le type de résultat est UInt8. -La fonction fonctionne également pour les chaînes. - -## notEmpty {#function-notempty} - -Retourne 0 pour un tableau vide, ou 1 pour un non-vide. -Le type de résultat est UInt8. -La fonction fonctionne également pour les chaînes. - -## longueur {#array_functions-length} - -Retourne le nombre d'éléments dans le tableau. -Le type de résultat est UInt64. -La fonction fonctionne également pour les chaînes. - -## emptyArrayUInt8, emptyArrayUInt16, emptyArrayUInt32, emptyArrayUInt64 {#emptyarrayuint8-emptyarrayuint16-emptyarrayuint32-emptyarrayuint64} - -## emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64 {#emptyarrayint8-emptyarrayint16-emptyarrayint32-emptyarrayint64} - -## emptyArrayFloat32, emptyArrayFloat64 {#emptyarrayfloat32-emptyarrayfloat64} - -## emptyArrayDate, emptyArrayDateTime {#emptyarraydate-emptyarraydatetime} - -## emptyArrayString {#emptyarraystring} - -Accepte zéro argument et renvoie un tableau vide du type approprié. - -## emptyArrayToSingle {#emptyarraytosingle} - -Accepte un tableau vide et renvoie un élément de tableau qui est égal à la valeur par défaut. - -## plage (fin), Plage(début, fin \[, étape\]) {#rangeend-rangestart-end-step} - -Retourne un tableau de nombres du début à la fin-1 par étape. -Si l'argument `start` n'est pas spécifié, la valeur par défaut est 0. -Si l'argument `step` n'est pas spécifié, la valeur par défaut est 1. -Il se comporte presque comme pythonic `range`. Mais la différence est que tous les types d'arguments doivent être `UInt` nombre. -Juste au cas où, une exception est levée si des tableaux d'une longueur totale de plus de 100 000 000 d'éléments sont créés dans un bloc de données. - -## array(x1, …), operator \[x1, …\] {#arrayx1-operator-x1} - -Crée un tableau à partir des arguments de la fonction. -Les arguments doivent être des constantes et avoir des types qui ont le plus petit type commun. Au moins un argument doit être passé, sinon il n'est pas clair quel type de tableau créer. Qui est, vous ne pouvez pas utiliser cette fonction pour créer un tableau vide (pour ce faire, utilisez la ‘emptyArray\*’ la fonction décrite ci-dessus). -Retourne un ‘Array(T)’ type de résultat, où ‘T’ est le plus petit type commun parmi les arguments passés. - -## arrayConcat {#arrayconcat} - -Combine des tableaux passés comme arguments. - -``` sql -arrayConcat(arrays) -``` - -**Paramètre** - -- `arrays` – Arbitrary number of arguments of [Tableau](../../sql-reference/data-types/array.md) type. - **Exemple** - - - -``` sql -SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res -``` - -``` text -┌─res───────────┐ -│ [1,2,3,4,5,6] │ -└───────────────┘ -``` - -## arrayElement(arr, n), opérateur arr\[n\] {#arrayelementarr-n-operator-arrn} - -Récupérer l'élément avec l'index `n` à partir du tableau `arr`. `n` doit être n'importe quel type entier. -Les index dans un tableau commencent à partir d'un. -Les index négatifs sont pris en charge. Dans ce cas, il sélectionne l'élément correspondant numérotées à partir de la fin. Exemple, `arr[-1]` est le dernier élément du tableau. - -Si l'index est en dehors des limites d'un tableau, il renvoie une valeur (0 pour les nombres, une chaîne vide pour les cordes, etc.), sauf pour le cas avec un tableau non constant et un index constant 0 (dans ce cas, il y aura une erreur `Array indices are 1-based`). - -## a (arr, elem) {#hasarr-elem} - -Vérifie si le ‘arr’ tableau a la ‘elem’ élément. -Retourne 0 si l'élément n'est pas dans le tableau, ou 1 si elle l'est. - -`NULL` est traitée comme une valeur. - -``` sql -SELECT has([1, 2, NULL], NULL) -``` - -``` text -┌─has([1, 2, NULL], NULL)─┐ -│ 1 │ -└─────────────────────────┘ -``` - -## hasAll {#hasall} - -Vérifie si un tableau est un sous-ensemble de l'autre. - -``` sql -hasAll(set, subset) -``` - -**Paramètre** - -- `set` – Array of any type with a set of elements. -- `subset` – Array of any type with elements that should be tested to be a subset of `set`. - -**Les valeurs de retour** - -- `1`, si `set` contient tous les éléments de `subset`. -- `0`, autrement. - -**Propriétés particulières** - -- Un tableau vide est un sous-ensemble d'un tableau quelconque. -- `Null` traitée comme une valeur. -- Ordre des valeurs dans les deux tableaux n'a pas d'importance. - -**Exemple** - -`SELECT hasAll([], [])` retours 1. - -`SELECT hasAll([1, Null], [Null])` retours 1. - -`SELECT hasAll([1.0, 2, 3, 4], [1, 3])` retours 1. - -`SELECT hasAll(['a', 'b'], ['a'])` retours 1. - -`SELECT hasAll([1], ['a'])` renvoie 0. - -`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` renvoie 0. - -## hasAny {#hasany} - -Vérifie si deux tableaux ont une intersection par certains éléments. - -``` sql -hasAny(array1, array2) -``` - -**Paramètre** - -- `array1` – Array of any type with a set of elements. -- `array2` – Array of any type with a set of elements. - -**Les valeurs de retour** - -- `1`, si `array1` et `array2` avoir un élément similaire au moins. -- `0`, autrement. - -**Propriétés particulières** - -- `Null` traitée comme une valeur. -- Ordre des valeurs dans les deux tableaux n'a pas d'importance. - -**Exemple** - -`SELECT hasAny([1], [])` retourner `0`. - -`SELECT hasAny([Null], [Null, 1])` retourner `1`. - -`SELECT hasAny([-128, 1., 512], [1])` retourner `1`. - -`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` retourner `0`. - -`SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` retourner `1`. - -## indexOf (arr, x) {#indexofarr-x} - -Renvoie l'index de la première ‘x’ élément (à partir de 1) s'il est dans le tableau, ou 0 s'il ne l'est pas. - -Exemple: - -``` sql -SELECT indexOf([1, 3, NULL, NULL], NULL) -``` - -``` text -┌─indexOf([1, 3, NULL, NULL], NULL)─┐ -│ 3 │ -└───────────────────────────────────┘ -``` - -Ensemble d'éléments de `NULL` sont traités comme des valeurs normales. - -## countEqual (arr, x) {#countequalarr-x} - -Renvoie le nombre d'éléments dans le tableau égal à X. équivalent à arrayCount (elem - \> elem = x, arr). - -`NULL` les éléments sont traités comme des valeurs distinctes. - -Exemple: - -``` sql -SELECT countEqual([1, 2, NULL, NULL], NULL) -``` - -``` text -┌─countEqual([1, 2, NULL, NULL], NULL)─┐ -│ 2 │ -└──────────────────────────────────────┘ -``` - -## arrayEnumerate (arr) {#array_functions-arrayenumerate} - -Returns the array \[1, 2, 3, …, length (arr) \] - -Cette fonction est normalement utilisée avec ARRAY JOIN. Il permet de compter quelque chose une seule fois pour chaque tableau après l'application de la jointure de tableau. Exemple: - -``` sql -SELECT - count() AS Reaches, - countIf(num = 1) AS Hits -FROM test.hits -ARRAY JOIN - GoalsReached, - arrayEnumerate(GoalsReached) AS num -WHERE CounterID = 160656 -LIMIT 10 -``` - -``` text -┌─Reaches─┬──Hits─┐ -│ 95606 │ 31406 │ -└─────────┴───────┘ -``` - -Dans cet exemple, Reaches est le nombre de conversions (les chaînes reçues après l'application de la jointure de tableau), et Hits est le nombre de pages vues (chaînes avant la jointure de tableau). Dans ce cas particulier, vous pouvez obtenir le même résultat dans une voie plus facile: - -``` sql -SELECT - sum(length(GoalsReached)) AS Reaches, - count() AS Hits -FROM test.hits -WHERE (CounterID = 160656) AND notEmpty(GoalsReached) -``` - -``` text -┌─Reaches─┬──Hits─┐ -│ 95606 │ 31406 │ -└─────────┴───────┘ -``` - -Cette fonction peut également être utilisée dans les fonctions d'ordre supérieur. Par exemple, vous pouvez l'utiliser pour obtenir les indices de tableau pour les éléments qui correspondent à une condition. - -## arrayEnumerateUniq(arr, …) {#arrayenumerateuniqarr} - -Renvoie un tableau de la même taille que le tableau source, indiquant pour chaque élément Quelle est sa position parmi les éléments de même valeur. -Par exemple: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. - -Cette fonction est utile lors de L'utilisation de la jointure de tableau et de l'agrégation d'éléments de tableau. -Exemple: - -``` sql -SELECT - Goals.ID AS GoalID, - sum(Sign) AS Reaches, - sumIf(Sign, num = 1) AS Visits -FROM test.visits -ARRAY JOIN - Goals, - arrayEnumerateUniq(Goals.ID) AS num -WHERE CounterID = 160656 -GROUP BY GoalID -ORDER BY Reaches DESC -LIMIT 10 -``` - -``` text -┌──GoalID─┬─Reaches─┬─Visits─┐ -│ 53225 │ 3214 │ 1097 │ -│ 2825062 │ 3188 │ 1097 │ -│ 56600 │ 2803 │ 488 │ -│ 1989037 │ 2401 │ 365 │ -│ 2830064 │ 2396 │ 910 │ -│ 1113562 │ 2372 │ 373 │ -│ 3270895 │ 2262 │ 812 │ -│ 1084657 │ 2262 │ 345 │ -│ 56599 │ 2260 │ 799 │ -│ 3271094 │ 2256 │ 812 │ -└─────────┴─────────┴────────┘ -``` - -Dans cet exemple, chaque ID d'objectif a un calcul du nombre de conversions (chaque élément de la structure de données imbriquées objectifs est un objectif atteint, que nous appelons une conversion) et le nombre de sessions. Sans array JOIN, nous aurions compté le nombre de sessions comme sum(signe). Mais dans ce cas particulier, les lignes ont été multipliées par la structure des objectifs imbriqués, donc pour compter chaque session une fois après cela, nous appliquons une condition à la valeur de arrayEnumerateUniq(Goals.ID) fonction. - -La fonction arrayEnumerateUniq peut prendre plusieurs tableaux de la même taille que les arguments. Dans ce cas, l'unicité est considérée pour les tuples d'éléments dans les mêmes positions dans tous les tableaux. - -``` sql -SELECT arrayEnumerateUniq([1, 1, 1, 2, 2, 2], [1, 1, 2, 1, 1, 2]) AS res -``` - -``` text -┌─res───────────┐ -│ [1,2,1,1,2,1] │ -└───────────────┘ -``` - -Ceci est nécessaire lors de L'utilisation de Array JOIN avec une structure de données imbriquée et une agrégation supplémentaire entre plusieurs éléments de cette structure. - -## arrayPopBack {#arraypopback} - -Supprime le dernier élément du tableau. - -``` sql -arrayPopBack(array) -``` - -**Paramètre** - -- `array` – Array. - -**Exemple** - -``` sql -SELECT arrayPopBack([1, 2, 3]) AS res -``` - -``` text -┌─res───┐ -│ [1,2] │ -└───────┘ -``` - -## arrayPopFront {#arraypopfront} - -Supprime le premier élément de la matrice. - -``` sql -arrayPopFront(array) -``` - -**Paramètre** - -- `array` – Array. - -**Exemple** - -``` sql -SELECT arrayPopFront([1, 2, 3]) AS res -``` - -``` text -┌─res───┐ -│ [2,3] │ -└───────┘ -``` - -## arrayPushBack {#arraypushback} - -Ajoute un élément à la fin du tableau. - -``` sql -arrayPushBack(array, single_value) -``` - -**Paramètre** - -- `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type pour le type de données du tableau. Pour plus d'informations sur les types de données dans ClickHouse, voir “[Types de données](../../sql-reference/data-types/index.md#data_types)”. Peut être `NULL`. La fonction ajoute un `NULL` tableau, et le type d'éléments de tableau convertit en `Nullable`. - -**Exemple** - -``` sql -SELECT arrayPushBack(['a'], 'b') AS res -``` - -``` text -┌─res───────┐ -│ ['a','b'] │ -└───────────┘ -``` - -## arrayPushFront {#arraypushfront} - -Ajoute un élément au début du tableau. - -``` sql -arrayPushFront(array, single_value) -``` - -**Paramètre** - -- `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type pour le type de données du tableau. Pour plus d'informations sur les types de données dans ClickHouse, voir “[Types de données](../../sql-reference/data-types/index.md#data_types)”. Peut être `NULL`. La fonction ajoute un `NULL` tableau, et le type d'éléments de tableau convertit en `Nullable`. - -**Exemple** - -``` sql -SELECT arrayPushFront(['b'], 'a') AS res -``` - -``` text -┌─res───────┐ -│ ['a','b'] │ -└───────────┘ -``` - -## arrayResize {#arrayresize} - -Les changements de la longueur du tableau. - -``` sql -arrayResize(array, size[, extender]) -``` - -**Paramètre:** - -- `array` — Array. -- `size` — Required length of the array. - - Si `size` est inférieure à la taille d'origine du tableau, le tableau est tronqué à partir de la droite. -- Si `size` est plus grande que la taille initiale du tableau, le tableau est étendu vers la droite avec `extender` valeurs ou valeurs par défaut pour le type de données des éléments du tableau. -- `extender` — Value for extending an array. Can be `NULL`. - -**Valeur renvoyée:** - -Un tableau de longueur `size`. - -**Exemples d'appels** - -``` sql -SELECT arrayResize([1], 3) -``` - -``` text -┌─arrayResize([1], 3)─┐ -│ [1,0,0] │ -└─────────────────────┘ -``` - -``` sql -SELECT arrayResize([1], 3, NULL) -``` - -``` text -┌─arrayResize([1], 3, NULL)─┐ -│ [1,NULL,NULL] │ -└───────────────────────────┘ -``` - -## arraySlice {#arrayslice} - -Retourne une tranche du tableau. - -``` sql -arraySlice(array, offset[, length]) -``` - -**Paramètre** - -- `array` – Array of data. -- `offset` – Indent from the edge of the array. A positive value indicates an offset on the left, and a negative value is an indent on the right. Numbering of the array items begins with 1. -- `length` - La longueur de la nécessaire tranche. Si vous spécifiez une valeur négative, la fonction renvoie un ouvert tranche `[offset, array_length - length)`. Si vous omettez la valeur, la fonction renvoie la tranche `[offset, the_end_of_array]`. - -**Exemple** - -``` sql -SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res -``` - -``` text -┌─res────────┐ -│ [2,NULL,4] │ -└────────────┘ -``` - -Éléments de tableau définis sur `NULL` sont traités comme des valeurs normales. - -## arraySort(\[func,\] arr, …) {#array_functions-sort} - -Trie les éléments de la `arr` tableau dans l'ordre croissant. Si l' `func` fonction est spécifiée, l'ordre de tri est déterminé par le résultat de la `func` fonction appliquée aux éléments du tableau. Si `func` accepte plusieurs arguments, le `arraySort` la fonction est passé plusieurs tableaux que les arguments de `func` correspond à. Des exemples détaillés sont présentés à la fin de `arraySort` Description. - -Exemple de tri de valeurs entières: - -``` sql -SELECT arraySort([1, 3, 3, 0]); -``` - -``` text -┌─arraySort([1, 3, 3, 0])─┐ -│ [0,1,3,3] │ -└─────────────────────────┘ -``` - -Exemple de tri des valeurs de chaîne: - -``` sql -SELECT arraySort(['hello', 'world', '!']); -``` - -``` text -┌─arraySort(['hello', 'world', '!'])─┐ -│ ['!','hello','world'] │ -└────────────────────────────────────┘ -``` - -Considérez l'ordre de tri suivant pour le `NULL`, `NaN` et `Inf` valeur: - -``` sql -SELECT arraySort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]); -``` - -``` text -┌─arraySort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf])─┐ -│ [-inf,-4,1,2,3,inf,nan,nan,NULL,NULL] │ -└───────────────────────────────────────────────────────────┘ -``` - -- `-Inf` les valeurs sont d'abord dans le tableau. -- `NULL` les valeurs sont les derniers dans le tableau. -- `NaN` les valeurs sont juste avant `NULL`. -- `Inf` les valeurs sont juste avant `NaN`. - -Notez que `arraySort` est un [fonction d'ordre supérieur](higher-order-functions.md). Vous pouvez passer d'une fonction lambda comme premier argument. Dans ce cas, l'ordre de classement est déterminé par le résultat de la fonction lambda appliquée aux éléments de la matrice. - -Considérons l'exemple suivant: - -``` sql -SELECT arraySort((x) -> -x, [1, 2, 3]) as res; -``` - -``` text -┌─res─────┐ -│ [3,2,1] │ -└─────────┘ -``` - -For each element of the source array, the lambda function returns the sorting key, that is, \[1 –\> -1, 2 –\> -2, 3 –\> -3\]. Since the `arraySort` fonction trie les touches dans l'ordre croissant, le résultat est \[3, 2, 1\]. Ainsi, l' `(x) –> -x` fonction lambda définit le [l'ordre décroissant](#array_functions-reverse-sort) dans un tri. - -La fonction lambda peut accepter plusieurs arguments. Dans ce cas, vous avez besoin de passer l' `arraySort` fonction plusieurs tableaux de longueur identique à laquelle correspondront les arguments de la fonction lambda. Le tableau résultant sera composé d'éléments du premier tableau d'entrée; les éléments du(des) Tableau (s) d'entrée suivant (s) spécifient les clés de tri. Exemple: - -``` sql -SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]) as res; -``` - -``` text -┌─res────────────────┐ -│ ['world', 'hello'] │ -└────────────────────┘ -``` - -Ici, les éléments qui sont passés dans le deuxième tableau (\[2, 1\]) définissent une clé de tri pour l'élément correspondant à partir du tableau source (\[‘hello’, ‘world’\]), qui est, \[‘hello’ –\> 2, ‘world’ –\> 1\]. Since the lambda function doesn't use `x`, les valeurs réelles du tableau source n'affectent pas l'ordre dans le résultat. Si, ‘hello’ sera le deuxième élément du résultat, et ‘world’ sera le premier. - -D'autres exemples sont présentés ci-dessous. - -``` sql -SELECT arraySort((x, y) -> y, [0, 1, 2], ['c', 'b', 'a']) as res; -``` - -``` text -┌─res─────┐ -│ [2,1,0] │ -└─────────┘ -``` - -``` sql -SELECT arraySort((x, y) -> -y, [0, 1, 2], [1, 2, 3]) as res; -``` - -``` text -┌─res─────┐ -│ [2,1,0] │ -└─────────┘ -``` - -!!! note "Note" - Pour améliorer l'efficacité du tri, de la [Transformation schwartzienne](https://en.wikipedia.org/wiki/Schwartzian_transform) est utilisée. - -## arrayReverseSort(\[func,\] arr, …) {#array_functions-reverse-sort} - -Trie les éléments de la `arr` tableau dans l'ordre décroissant. Si l' `func` la fonction est spécifiée, `arr` est trié en fonction du résultat de la `func` fonction appliquée aux éléments du tableau, puis le tableau trié est inversé. Si `func` accepte plusieurs arguments, le `arrayReverseSort` la fonction est passé plusieurs tableaux que les arguments de `func` correspond à. Des exemples détaillés sont présentés à la fin de `arrayReverseSort` Description. - -Exemple de tri de valeurs entières: - -``` sql -SELECT arrayReverseSort([1, 3, 3, 0]); -``` - -``` text -┌─arrayReverseSort([1, 3, 3, 0])─┐ -│ [3,3,1,0] │ -└────────────────────────────────┘ -``` - -Exemple de tri des valeurs de chaîne: - -``` sql -SELECT arrayReverseSort(['hello', 'world', '!']); -``` - -``` text -┌─arrayReverseSort(['hello', 'world', '!'])─┐ -│ ['world','hello','!'] │ -└───────────────────────────────────────────┘ -``` - -Considérez l'ordre de tri suivant pour le `NULL`, `NaN` et `Inf` valeur: - -``` sql -SELECT arrayReverseSort([1, nan, 2, NULL, 3, nan, -4, NULL, inf, -inf]) as res; -``` - -``` text -┌─res───────────────────────────────────┐ -│ [inf,3,2,1,-4,-inf,nan,nan,NULL,NULL] │ -└───────────────────────────────────────┘ -``` - -- `Inf` les valeurs sont d'abord dans le tableau. -- `NULL` les valeurs sont les derniers dans le tableau. -- `NaN` les valeurs sont juste avant `NULL`. -- `-Inf` les valeurs sont juste avant `NaN`. - -Notez que l' `arrayReverseSort` est un [fonction d'ordre supérieur](higher-order-functions.md). Vous pouvez passer d'une fonction lambda comme premier argument. Exemple est montré ci-dessous. - -``` sql -SELECT arrayReverseSort((x) -> -x, [1, 2, 3]) as res; -``` - -``` text -┌─res─────┐ -│ [1,2,3] │ -└─────────┘ -``` - -Le tableau est trié de la façon suivante: - -1. Dans un premier temps, le tableau source (\[1, 2, 3\]) est trié en fonction du résultat de la fonction lambda appliquée aux éléments du tableau. Le résultat est un tableau \[3, 2, 1\]. -2. Tableau qui est obtenu à l'étape précédente, est renversé. Donc, le résultat final est \[1, 2, 3\]. - -La fonction lambda peut accepter plusieurs arguments. Dans ce cas, vous avez besoin de passer l' `arrayReverseSort` fonction plusieurs tableaux de longueur identique à laquelle correspondront les arguments de la fonction lambda. Le tableau résultant sera composé d'éléments du premier tableau d'entrée; les éléments du(des) Tableau (s) d'entrée suivant (s) spécifient les clés de tri. Exemple: - -``` sql -SELECT arrayReverseSort((x, y) -> y, ['hello', 'world'], [2, 1]) as res; -``` - -``` text -┌─res───────────────┐ -│ ['hello','world'] │ -└───────────────────┘ -``` - -Dans cet exemple, le tableau est trié de la façon suivante: - -1. Au début, le tableau source (\[‘hello’, ‘world’\]) est triée selon le résultat de la fonction lambda appliquée aux éléments de tableaux. Les éléments qui sont passés dans le deuxième tableau (\[2, 1\]), définissent les clés de tri pour les éléments correspondants du tableau source. Le résultat est un tableau \[‘world’, ‘hello’\]. -2. Tableau trié lors de l'étape précédente, est renversé. Donc, le résultat final est \[‘hello’, ‘world’\]. - -D'autres exemples sont présentés ci-dessous. - -``` sql -SELECT arrayReverseSort((x, y) -> y, [4, 3, 5], ['a', 'b', 'c']) AS res; -``` - -``` text -┌─res─────┐ -│ [5,3,4] │ -└─────────┘ -``` - -``` sql -SELECT arrayReverseSort((x, y) -> -y, [4, 3, 5], [1, 2, 3]) AS res; -``` - -``` text -┌─res─────┐ -│ [4,3,5] │ -└─────────┘ -``` - -## arrayUniq(arr, …) {#arrayuniqarr} - -Si un argument est passé, il compte le nombre de différents éléments dans le tableau. -Si plusieurs arguments sont passés, il compte le nombre de tuples différents d'éléments aux positions correspondantes dans plusieurs tableaux. - -Si vous souhaitez obtenir une liste des éléments dans un tableau, vous pouvez utiliser arrayReduce(‘groupUniqArray’, arr). - -## arrayJoin (arr) {#array-functions-join} - -Une fonction spéciale. Voir la section [“ArrayJoin function”](array-join.md#functions_arrayjoin). - -## tableaudifférence {#arraydifference} - -Calcule la différence entre les éléments de tableau adjacents. Renvoie un tableau où le premier élément sera 0, le second est la différence entre `a[1] - a[0]`, etc. The type of elements in the resulting array is determined by the type inference rules for subtraction (e.g. `UInt8` - `UInt8` = `Int16`). - -**Syntaxe** - -``` sql -arrayDifference(array) -``` - -**Paramètre** - -- `array` – [Tableau](https://clickhouse.tech/docs/en/data_types/array/). - -**Valeurs renvoyées** - -Renvoie un tableau de différences entre les éléments adjacents. - -Type: [UInt\*](https://clickhouse.tech/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.tech/docs/en/data_types/int_uint/#int-ranges), [Flottant\*](https://clickhouse.tech/docs/en/data_types/float/). - -**Exemple** - -Requête: - -``` sql -SELECT arrayDifference([1, 2, 3, 4]) -``` - -Résultat: - -``` text -┌─arrayDifference([1, 2, 3, 4])─┐ -│ [0,1,1,1] │ -└───────────────────────────────┘ -``` - -Exemple de débordement dû au type de résultat Int64: - -Requête: - -``` sql -SELECT arrayDifference([0, 10000000000000000000]) -``` - -Résultat: - -``` text -┌─arrayDifference([0, 10000000000000000000])─┐ -│ [0,-8446744073709551616] │ -└────────────────────────────────────────────┘ -``` - -## arrayDistinct {#arraydistinct} - -Prend un tableau, retourne un tableau contenant les différents éléments seulement. - -**Syntaxe** - -``` sql -arrayDistinct(array) -``` - -**Paramètre** - -- `array` – [Tableau](https://clickhouse.tech/docs/en/data_types/array/). - -**Valeurs renvoyées** - -Retourne un tableau contenant les éléments distincts. - -**Exemple** - -Requête: - -``` sql -SELECT arrayDistinct([1, 2, 2, 3, 1]) -``` - -Résultat: - -``` text -┌─arrayDistinct([1, 2, 2, 3, 1])─┐ -│ [1,2,3] │ -└────────────────────────────────┘ -``` - -## arrayEnumerateDense(arr) {#array_functions-arrayenumeratedense} - -Renvoie un tableau de la même taille que le tableau source, indiquant où chaque élément apparaît en premier dans le tableau source. - -Exemple: - -``` sql -SELECT arrayEnumerateDense([10, 20, 10, 30]) -``` - -``` text -┌─arrayEnumerateDense([10, 20, 10, 30])─┐ -│ [1,2,1,3] │ -└───────────────────────────────────────┘ -``` - -## arrayIntersect (arr) {#array-functions-arrayintersect} - -Prend plusieurs tableaux, retourne un tableau avec des éléments présents dans tous les tableaux source. L'ordre des éléments dans le tableau résultant est le même que dans le premier tableau. - -Exemple: - -``` sql -SELECT - arrayIntersect([1, 2], [1, 3], [2, 3]) AS no_intersect, - arrayIntersect([1, 2], [1, 3], [1, 4]) AS intersect -``` - -``` text -┌─no_intersect─┬─intersect─┐ -│ [] │ [1] │ -└──────────────┴───────────┘ -``` - -## arrayReduce {#arrayreduce} - -Applique une fonction d'agrégation aux éléments du tableau et renvoie son résultat. Le nom de la fonction d'agrégation est passé sous forme de chaîne entre guillemets simples `'max'`, `'sum'`. Lorsque vous utilisez des fonctions d'agrégat paramétriques, le paramètre est indiqué après le nom de la fonction entre parenthèses `'uniqUpTo(6)'`. - -**Syntaxe** - -``` sql -arrayReduce(agg_func, arr1, arr2, ..., arrN) -``` - -**Paramètre** - -- `agg_func` — The name of an aggregate function which should be a constant [chaîne](../../sql-reference/data-types/string.md). -- `arr` — Any number of [tableau](../../sql-reference/data-types/array.md) tapez les colonnes comme paramètres de la fonction d'agrégation. - -**Valeur renvoyée** - -**Exemple** - -``` sql -SELECT arrayReduce('max', [1, 2, 3]) -``` - -``` text -┌─arrayReduce('max', [1, 2, 3])─┐ -│ 3 │ -└───────────────────────────────┘ -``` - -Si une fonction d'agrégation prend plusieurs arguments, cette fonction doit être appliqué à plusieurs ensembles de même taille. - -``` sql -SELECT arrayReduce('maxIf', [3, 5], [1, 0]) -``` - -``` text -┌─arrayReduce('maxIf', [3, 5], [1, 0])─┐ -│ 3 │ -└──────────────────────────────────────┘ -``` - -Exemple avec une fonction d'agrégat paramétrique: - -``` sql -SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) -``` - -``` text -┌─arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10])─┐ -│ 4 │ -└─────────────────────────────────────────────────────────────┘ -``` - -## arrayReduceInRanges {#arrayreduceinranges} - -Applique une fonction d'agrégation d'éléments de tableau dans des plages et retourne un tableau contenant le résultat correspondant à chaque gamme. La fonction retourne le même résultat que plusieurs `arrayReduce(agg_func, arraySlice(arr1, index, length), ...)`. - -**Syntaxe** - -``` sql -arrayReduceInRanges(agg_func, ranges, arr1, arr2, ..., arrN) -``` - -**Paramètre** - -- `agg_func` — The name of an aggregate function which should be a constant [chaîne](../../sql-reference/data-types/string.md). -- `ranges` — The ranges to aggretate which should be an [tableau](../../sql-reference/data-types/array.md) de [tuple](../../sql-reference/data-types/tuple.md) qui contient l'indice et la longueur de chaque plage. -- `arr` — Any number of [tableau](../../sql-reference/data-types/array.md) tapez les colonnes comme paramètres de la fonction d'agrégation. - -**Valeur renvoyée** - -**Exemple** - -``` sql -SELECT arrayReduceInRanges( - 'sum', - [(1, 5), (2, 3), (3, 4), (4, 4)], - [1000000, 200000, 30000, 4000, 500, 60, 7] -) AS res -``` - -``` text -┌─res─────────────────────────┐ -│ [1234500,234000,34560,4567] │ -└─────────────────────────────┘ -``` - -## arrayReverse(arr) {#arrayreverse} - -Retourne un tableau de la même taille que l'original tableau contenant les éléments dans l'ordre inverse. - -Exemple: - -``` sql -SELECT arrayReverse([1, 2, 3]) -``` - -``` text -┌─arrayReverse([1, 2, 3])─┐ -│ [3,2,1] │ -└─────────────────────────┘ -``` - -## inverse (arr) {#array-functions-reverse} - -Synonyme de [“arrayReverse”](#arrayreverse) - -## arrayFlatten {#arrayflatten} - -Convertit un tableau de tableaux dans un tableau associatif. - -Fonction: - -- S'applique à toute profondeur de tableaux imbriqués. -- Ne change pas les tableaux qui sont déjà plats. - -Le tableau aplati contient tous les éléments de tous les tableaux source. - -**Syntaxe** - -``` sql -flatten(array_of_arrays) -``` - -Alias: `flatten`. - -**Paramètre** - -- `array_of_arrays` — [Tableau](../../sql-reference/data-types/array.md) de tableaux. Exemple, `[[1,2,3], [4,5]]`. - -**Exemple** - -``` sql -SELECT flatten([[[1]], [[2], [3]]]) -``` - -``` text -┌─flatten(array(array([1]), array([2], [3])))─┐ -│ [1,2,3] │ -└─────────────────────────────────────────────┘ -``` - -## arrayCompact {#arraycompact} - -Supprime les éléments en double consécutifs d'un tableau. L'ordre des valeurs de résultat est déterminée par l'ordre dans le tableau source. - -**Syntaxe** - -``` sql -arrayCompact(arr) -``` - -**Paramètre** - -`arr` — The [tableau](../../sql-reference/data-types/array.md) inspecter. - -**Valeur renvoyée** - -Le tableau sans doublon. - -Type: `Array`. - -**Exemple** - -Requête: - -``` sql -SELECT arrayCompact([1, 1, nan, nan, 2, 3, 3, 3]) -``` - -Résultat: - -``` text -┌─arrayCompact([1, 1, nan, nan, 2, 3, 3, 3])─┐ -│ [1,nan,nan,2,3] │ -└────────────────────────────────────────────┘ -``` - -## arrayZip {#arrayzip} - -Combine plusieurs tableaux en un seul tableau. Le tableau résultant contient les éléments correspondants des tableaux source regroupés en tuples dans l'ordre des arguments listés. - -**Syntaxe** - -``` sql -arrayZip(arr1, arr2, ..., arrN) -``` - -**Paramètre** - -- `arrN` — [Tableau](../data-types/array.md). - -La fonction peut prendre n'importe quel nombre de tableaux de différents types. Tous les tableaux doivent être de taille égale. - -**Valeur renvoyée** - -- Tableau avec des éléments des tableaux source regroupés en [tuple](../data-types/tuple.md). Types de données dans le tuple sont les mêmes que les types de l'entrée des tableaux et dans le même ordre que les tableaux sont passés. - -Type: [Tableau](../data-types/array.md). - -**Exemple** - -Requête: - -``` sql -SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]) -``` - -Résultat: - -``` text -┌─arrayZip(['a', 'b', 'c'], [5, 2, 1])─┐ -│ [('a',5),('b',2),('c',1)] │ -└──────────────────────────────────────┘ -``` - -## arrayAUC {#arrayauc} - -Calculer AUC (zone sous la courbe, qui est un concept dans l'apprentissage automatique, voir plus de détails: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). - -**Syntaxe** - -``` sql -arrayAUC(arr_scores, arr_labels) -``` - -**Paramètre** -- `arr_scores` — scores prediction model gives. -- `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negtive sample. - -**Valeur renvoyée** -Renvoie la valeur AUC avec le type Float64. - -**Exemple** -Requête: - -``` sql -select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) -``` - -Résultat: - -``` text -┌─arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ -│ 0.75 │ -└────────────────────────────────────────---──┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/array_functions/) diff --git a/docs/fr/sql-reference/functions/array-join.md b/docs/fr/sql-reference/functions/array-join.md deleted file mode 100644 index 859e801994d..00000000000 --- a/docs/fr/sql-reference/functions/array-join.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 61 -toc_title: arrayJoin ---- - -# fonction arrayJoin {#functions_arrayjoin} - -C'est un très inhabituelle de la fonction. - -Les fonctions normales ne modifient pas un ensemble de lignes, mais modifient simplement les valeurs de chaque ligne (map). -Les fonctions d'agrégation compriment un ensemble de lignes (plier ou réduire). -Le ‘arrayJoin’ la fonction prend chaque ligne et génère un ensemble de lignes (dépliante). - -Cette fonction prend un tableau comme argument et propage la ligne source à plusieurs lignes pour le nombre d'éléments dans le tableau. -Toutes les valeurs des colonnes sont simplement copiés, sauf les valeurs dans la colonne où cette fonction est appliquée; elle est remplacée par la valeur correspondante de tableau. - -Une requête peut utiliser plusieurs `arrayJoin` fonction. Dans ce cas, la transformation est effectuée plusieurs fois. - -Notez la syntaxe de jointure de tableau dans la requête SELECT, qui offre des possibilités plus larges. - -Exemple: - -``` sql -SELECT arrayJoin([1, 2, 3] AS src) AS dst, 'Hello', src -``` - -``` text -┌─dst─┬─\'Hello\'─┬─src─────┐ -│ 1 │ Hello │ [1,2,3] │ -│ 2 │ Hello │ [1,2,3] │ -│ 3 │ Hello │ [1,2,3] │ -└─────┴───────────┴─────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/array_join/) diff --git a/docs/fr/sql-reference/functions/bit-functions.md b/docs/fr/sql-reference/functions/bit-functions.md deleted file mode 100644 index 7b8795815f2..00000000000 --- a/docs/fr/sql-reference/functions/bit-functions.md +++ /dev/null @@ -1,255 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 48 -toc_title: Bit ---- - -# Peu De Fonctions {#bit-functions} - -Les fonctions Bit fonctionnent pour n'importe quelle paire de types de UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32 ou Float64. - -Le type de résultat est un entier avec des bits égaux aux bits maximum de ses arguments. Si au moins l'un des arguments est signé, le résultat est un signé nombre. Si un argument est un nombre à virgule flottante, Il est converti en Int64. - -## bitAnd (a, b) {#bitanda-b} - -## bitOr (a, b) {#bitora-b} - -## bitXor (a, b) {#bitxora-b} - -## bitNot (a) {#bitnota} - -## bitShiftLeft (A, b) {#bitshiftlefta-b} - -## bitShiftRight (A, b) {#bitshiftrighta-b} - -## bitRotateLeft (a, b) {#bitrotatelefta-b} - -## bitRotateRight (a, b) {#bitrotaterighta-b} - -## bitTest {#bittest} - -Prend tout entier et le convertit en [forme binaire](https://en.wikipedia.org/wiki/Binary_number) renvoie la valeur d'un bit à la position spécifiée. Le compte à rebours commence à partir de 0 de la droite vers la gauche. - -**Syntaxe** - -``` sql -SELECT bitTest(number, index) -``` - -**Paramètre** - -- `number` – integer number. -- `index` – position of bit. - -**Valeurs renvoyées** - -Renvoie une valeur de bit à la position spécifiée. - -Type: `UInt8`. - -**Exemple** - -Par exemple, le nombre 43 dans le système numérique de base-2 (binaire) est 101011. - -Requête: - -``` sql -SELECT bitTest(43, 1) -``` - -Résultat: - -``` text -┌─bitTest(43, 1)─┐ -│ 1 │ -└────────────────┘ -``` - -Un autre exemple: - -Requête: - -``` sql -SELECT bitTest(43, 2) -``` - -Résultat: - -``` text -┌─bitTest(43, 2)─┐ -│ 0 │ -└────────────────┘ -``` - -## bitTestAll {#bittestall} - -Renvoie le résultat de [logique de conjonction](https://en.wikipedia.org/wiki/Logical_conjunction) (Et opérateur) de tous les bits à des positions données. Le compte à rebours commence à partir de 0 de la droite vers la gauche. - -La conjonction pour les opérations bit à bit: - -0 AND 0 = 0 - -0 AND 1 = 0 - -1 AND 0 = 0 - -1 AND 1 = 1 - -**Syntaxe** - -``` sql -SELECT bitTestAll(number, index1, index2, index3, index4, ...) -``` - -**Paramètre** - -- `number` – integer number. -- `index1`, `index2`, `index3`, `index4` – positions of bit. For example, for set of positions (`index1`, `index2`, `index3`, `index4`) est vrai si et seulement si toutes ses positions sont remplies (`index1` ⋀ `index2`, ⋀ `index3` ⋀ `index4`). - -**Valeurs renvoyées** - -Retourne le résultat de la conjonction logique. - -Type: `UInt8`. - -**Exemple** - -Par exemple, le nombre 43 dans le système numérique de base-2 (binaire) est 101011. - -Requête: - -``` sql -SELECT bitTestAll(43, 0, 1, 3, 5) -``` - -Résultat: - -``` text -┌─bitTestAll(43, 0, 1, 3, 5)─┐ -│ 1 │ -└────────────────────────────┘ -``` - -Un autre exemple: - -Requête: - -``` sql -SELECT bitTestAll(43, 0, 1, 3, 5, 2) -``` - -Résultat: - -``` text -┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐ -│ 0 │ -└───────────────────────────────┘ -``` - -## bitTestAny {#bittestany} - -Renvoie le résultat de [disjonction logique](https://en.wikipedia.org/wiki/Logical_disjunction) (Ou opérateur) de tous les bits à des positions données. Le compte à rebours commence à partir de 0 de la droite vers la gauche. - -La disjonction pour les opérations binaires: - -0 OR 0 = 0 - -0 OR 1 = 1 - -1 OR 0 = 1 - -1 OR 1 = 1 - -**Syntaxe** - -``` sql -SELECT bitTestAny(number, index1, index2, index3, index4, ...) -``` - -**Paramètre** - -- `number` – integer number. -- `index1`, `index2`, `index3`, `index4` – positions of bit. - -**Valeurs renvoyées** - -Renvoie le résultat de la disjuction logique. - -Type: `UInt8`. - -**Exemple** - -Par exemple, le nombre 43 dans le système numérique de base-2 (binaire) est 101011. - -Requête: - -``` sql -SELECT bitTestAny(43, 0, 2) -``` - -Résultat: - -``` text -┌─bitTestAny(43, 0, 2)─┐ -│ 1 │ -└──────────────────────┘ -``` - -Un autre exemple: - -Requête: - -``` sql -SELECT bitTestAny(43, 4, 2) -``` - -Résultat: - -``` text -┌─bitTestAny(43, 4, 2)─┐ -│ 0 │ -└──────────────────────┘ -``` - -## bitCount {#bitcount} - -Calcule le nombre de bits mis à un dans la représentation binaire d'un nombre. - -**Syntaxe** - -``` sql -bitCount(x) -``` - -**Paramètre** - -- `x` — [Entier](../../sql-reference/data-types/int-uint.md) ou [virgule flottante](../../sql-reference/data-types/float.md) nombre. La fonction utilise la représentation de la valeur en mémoire. Il permet de financer les nombres à virgule flottante. - -**Valeur renvoyée** - -- Nombre de bits défini sur un dans le numéro d'entrée. - -La fonction ne convertit pas la valeur d'entrée en un type plus grand ([l'extension du signe](https://en.wikipedia.org/wiki/Sign_extension)). Ainsi, par exemple, `bitCount(toUInt8(-1)) = 8`. - -Type: `UInt8`. - -**Exemple** - -Prenez par exemple le numéro 333. Sa représentation binaire: 0000000101001101. - -Requête: - -``` sql -SELECT bitCount(333) -``` - -Résultat: - -``` text -┌─bitCount(333)─┐ -│ 5 │ -└───────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/bit_functions/) diff --git a/docs/fr/sql-reference/functions/bitmap-functions.md b/docs/fr/sql-reference/functions/bitmap-functions.md deleted file mode 100644 index 15cb68ffc52..00000000000 --- a/docs/fr/sql-reference/functions/bitmap-functions.md +++ /dev/null @@ -1,496 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 49 -toc_title: Bitmap ---- - -# Fonctions De Bitmap {#bitmap-functions} - -Les fonctions Bitmap fonctionnent pour le calcul de la valeur de L'objet de deux bitmaps, il s'agit de renvoyer un nouveau bitmap ou une cardinalité tout en utilisant le calcul de la formule, tel que and, or, xor, and not, etc. - -Il existe 2 types de méthodes de construction pour L'objet Bitmap. L'un doit être construit par la fonction d'agrégation groupBitmap avec-State, l'autre doit être construit par L'objet Array. Il est également de convertir L'objet Bitmap en objet tableau. - -RoaringBitmap est enveloppé dans une structure de données pendant le stockage réel des objets Bitmap. Lorsque la cardinalité est inférieure ou égale à 32, elle utilise Set objet. Lorsque la cardinalité est supérieure à 32, elle utilise l'objet RoaringBitmap. C'est pourquoi le stockage de faible cardinalité jeu est plus rapide. - -Pour plus d'informations sur RoaringBitmap, voir: [CRoaring](https://github.com/RoaringBitmap/CRoaring). - -## bitmapBuild {#bitmap_functions-bitmapbuild} - -Construire un bitmap à partir d'un tableau entier non signé. - -``` sql -bitmapBuild(array) -``` - -**Paramètre** - -- `array` – unsigned integer array. - -**Exemple** - -``` sql -SELECT bitmapBuild([1, 2, 3, 4, 5]) AS res, toTypeName(res) -``` - -``` text -┌─res─┬─toTypeName(bitmapBuild([1, 2, 3, 4, 5]))─────┐ -│  │ AggregateFunction(groupBitmap, UInt8) │ -└─────┴──────────────────────────────────────────────┘ -``` - -## bitmapToArray {#bitmaptoarray} - -Convertir bitmap en tableau entier. - -``` sql -bitmapToArray(bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res -``` - -``` text -┌─res─────────┐ -│ [1,2,3,4,5] │ -└─────────────┘ -``` - -## bitmapSubsetInRange {#bitmap-functions-bitmapsubsetinrange} - -Retourne le sous-ensemble dans la plage spécifiée (n'inclut pas le range_end). - -``` sql -bitmapSubsetInRange(bitmap, range_start, range_end) -``` - -**Paramètre** - -- `bitmap` – [Objet Bitmap](#bitmap_functions-bitmapbuild). -- `range_start` – range start point. Type: [UInt32](../../sql-reference/data-types/int-uint.md). -- `range_end` – range end point(excluded). Type: [UInt32](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res -``` - -``` text -┌─res───────────────┐ -│ [30,31,32,33,100] │ -└───────────────────┘ -``` - -## bitmapSubsetLimit {#bitmapsubsetlimit} - -Crée un sous-ensemble de bitmap avec n éléments pris entre `range_start` et `cardinality_limit`. - -**Syntaxe** - -``` sql -bitmapSubsetLimit(bitmap, range_start, cardinality_limit) -``` - -**Paramètre** - -- `bitmap` – [Objet Bitmap](#bitmap_functions-bitmapbuild). -- `range_start` – The subset starting point. Type: [UInt32](../../sql-reference/data-types/int-uint.md). -- `cardinality_limit` – The subset cardinality upper limit. Type: [UInt32](../../sql-reference/data-types/int-uint.md). - -**Valeur renvoyée** - -Ensemble. - -Type: `Bitmap object`. - -**Exemple** - -Requête: - -``` sql -SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res -``` - -Résultat: - -``` text -┌─res───────────────────────┐ -│ [30,31,32,33,100,200,500] │ -└───────────────────────────┘ -``` - -## bitmapContains {#bitmap_functions-bitmapcontains} - -Vérifie si le bitmap contient un élément. - -``` sql -bitmapContains(haystack, needle) -``` - -**Paramètre** - -- `haystack` – [Objet Bitmap](#bitmap_functions-bitmapbuild) où la fonction recherche. -- `needle` – Value that the function searches. Type: [UInt32](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- 0 — If `haystack` ne contient pas de `needle`. -- 1 — If `haystack` contenir `needle`. - -Type: `UInt8`. - -**Exemple** - -``` sql -SELECT bitmapContains(bitmapBuild([1,5,7,9]), toUInt32(9)) AS res -``` - -``` text -┌─res─┐ -│ 1 │ -└─────┘ -``` - -## bitmapHasAny {#bitmaphasany} - -Vérifie si deux bitmaps ont une intersection par certains éléments. - -``` sql -bitmapHasAny(bitmap1, bitmap2) -``` - -Si vous êtes sûr que `bitmap2` contient strictement un élément, envisagez d'utiliser le [bitmapContains](#bitmap_functions-bitmapcontains) fonction. Cela fonctionne plus efficacement. - -**Paramètre** - -- `bitmap*` – bitmap object. - -**Les valeurs de retour** - -- `1`, si `bitmap1` et `bitmap2` avoir un élément similaire au moins. -- `0`, autrement. - -**Exemple** - -``` sql -SELECT bitmapHasAny(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res -``` - -``` text -┌─res─┐ -│ 1 │ -└─────┘ -``` - -## bitmapHasAll {#bitmaphasall} - -Analogue à `hasAll(array, array)` renvoie 1 si le premier bitmap contient tous les éléments du second, 0 sinon. -Si le deuxième argument est un bitmap vide, alors renvoie 1. - -``` sql -bitmapHasAll(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapHasAll(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res -``` - -``` text -┌─res─┐ -│ 0 │ -└─────┘ -``` - -## bitmapCardinality {#bitmapcardinality} - -Retrun bitmap cardinalité de type UInt64. - -``` sql -bitmapCardinality(bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res -``` - -``` text -┌─res─┐ -│ 5 │ -└─────┘ -``` - -## bitmapMin {#bitmapmin} - -Retrun la plus petite valeur de type UInt64 dans l'ensemble, UINT32_MAX si l'ensemble est vide. - - bitmapMin(bitmap) - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res -``` - - ┌─res─┐ - │ 1 │ - └─────┘ - -## bitmapMax {#bitmapmax} - -Retrun la plus grande valeur de type UInt64 dans l'ensemble, 0 si l'ensemble est vide. - - bitmapMax(bitmap) - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res -``` - - ┌─res─┐ - │ 5 │ - └─────┘ - -## bitmapTransform {#bitmaptransform} - -Transformer un tableau de valeurs d'une image à l'autre tableau de valeurs, le résultat est une nouvelle image. - - bitmapTransform(bitmap, from_array, to_array) - -**Paramètre** - -- `bitmap` – bitmap object. -- `from_array` – UInt32 array. For idx in range \[0, from_array.size()), if bitmap contains from_array\[idx\], then replace it with to_array\[idx\]. Note that the result depends on array ordering if there are common elements between from_array and to_array. -- `to_array` – UInt32 array, its size shall be the same to from_array. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapTransform(bitmapBuild([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), cast([5,999,2] as Array(UInt32)), cast([2,888,20] as Array(UInt32)))) AS res -``` - - ┌─res───────────────────┐ - │ [1,3,4,6,7,8,9,10,20] │ - └───────────────────────┘ - -## bitmapAnd {#bitmapand} - -Deux bitmap et calcul, le résultat est un nouveau bitmap. - -``` sql -bitmapAnd(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapAnd(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res -``` - -``` text -┌─res─┐ -│ [3] │ -└─────┘ -``` - -## bitmapOr {#bitmapor} - -Deux bitmap ou calcul, le résultat est un nouveau bitmap. - -``` sql -bitmapOr(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapOr(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res -``` - -``` text -┌─res─────────┐ -│ [1,2,3,4,5] │ -└─────────────┘ -``` - -## bitmapXor {#bitmapxor} - -Deux bitmap xor calcul, le résultat est une nouvelle image. - -``` sql -bitmapXor(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapXor(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res -``` - -``` text -┌─res───────┐ -│ [1,2,4,5] │ -└───────────┘ -``` - -## bitmapetnot {#bitmapandnot} - -Deux Bitmap andnot calcul, le résultat est un nouveau bitmap. - -``` sql -bitmapAndnot(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapToArray(bitmapAndnot(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res -``` - -``` text -┌─res───┐ -│ [1,2] │ -└───────┘ -``` - -## bitmapetcardinalité {#bitmapandcardinality} - -Deux bitmap et calcul, retour cardinalité de type UInt64. - -``` sql -bitmapAndCardinality(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapAndCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res; -``` - -``` text -┌─res─┐ -│ 1 │ -└─────┘ -``` - -## bitmapOrCardinality {#bitmaporcardinality} - -Deux bitmap ou calcul, retour cardinalité de type UInt64. - -``` sql -bitmapOrCardinality(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapOrCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res; -``` - -``` text -┌─res─┐ -│ 5 │ -└─────┘ -``` - -## bitmapXorCardinality {#bitmapxorcardinality} - -Deux bitmap XOR calcul, retour cardinalité de type UInt64. - -``` sql -bitmapXorCardinality(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapXorCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res; -``` - -``` text -┌─res─┐ -│ 4 │ -└─────┘ -``` - -## bitmapetnotcardinality {#bitmapandnotcardinality} - -Deux bitmap andnot calcul, retour cardinalité de type UInt64. - -``` sql -bitmapAndnotCardinality(bitmap,bitmap) -``` - -**Paramètre** - -- `bitmap` – bitmap object. - -**Exemple** - -``` sql -SELECT bitmapAndnotCardinality(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res; -``` - -``` text -┌─res─┐ -│ 2 │ -└─────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/bitmap_functions/) diff --git a/docs/fr/sql-reference/functions/comparison-functions.md b/docs/fr/sql-reference/functions/comparison-functions.md deleted file mode 100644 index a5008c676fa..00000000000 --- a/docs/fr/sql-reference/functions/comparison-functions.md +++ /dev/null @@ -1,37 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: Comparaison ---- - -# Fonctions De Comparaison {#comparison-functions} - -Les fonctions de comparaison renvoient toujours 0 ou 1 (Uint8). - -Les types suivants peuvent être comparés: - -- nombre -- cordes et cordes fixes -- date -- dates avec heures - -au sein de chaque groupe, mais pas entre différents groupes. - -Par exemple, vous ne pouvez pas comparer une date avec une chaîne. Vous devez utiliser une fonction pour convertir la chaîne en une date, ou vice versa. - -Les chaînes sont comparées par octets. Une courte chaîne est plus petite que toutes les chaînes qui commencent par elle et qui contiennent au moins un caractère de plus. - -## égal, A = B et a = = b opérateur {#function-equals} - -## notEquals, a ! opérateur= b et a \<\> b {#function-notequals} - -## moins, opérateur \< {#function-less} - -## de plus, \> opérateur {#function-greater} - -## lessOrEquals, \< = opérateur {#function-lessorequals} - -## greaterOrEquals, \> = opérateur {#function-greaterorequals} - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/comparison_functions/) diff --git a/docs/fr/sql-reference/functions/conditional-functions.md b/docs/fr/sql-reference/functions/conditional-functions.md deleted file mode 100644 index 3912b49aa6a..00000000000 --- a/docs/fr/sql-reference/functions/conditional-functions.md +++ /dev/null @@ -1,207 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 43 -toc_title: 'Conditionnel ' ---- - -# Fonctions Conditionnelles {#conditional-functions} - -## si {#if} - -Contrôle la ramification conditionnelle. Contrairement à la plupart des systèmes, ClickHouse évalue toujours les deux expressions `then` et `else`. - -**Syntaxe** - -``` sql -SELECT if(cond, then, else) -``` - -Si la condition `cond` renvoie une valeur non nulle, retourne le résultat de l'expression `then` et le résultat de l'expression `else`, si présent, est ignoré. Si l' `cond` est égal à zéro ou `NULL` alors le résultat de la `then` l'expression est ignorée et le résultat de `else` expression, si elle est présente, est renvoyée. - -**Paramètre** - -- `cond` – The condition for evaluation that can be zero or not. The type is UInt8, Nullable(UInt8) or NULL. -- `then` - L'expression à renvoyer si la condition est remplie. -- `else` - L'expression à renvoyer si la condition n'est pas remplie. - -**Valeurs renvoyées** - -La fonction s'exécute `then` et `else` expressions et retourne son résultat, selon que la condition `cond` fini par être zéro ou pas. - -**Exemple** - -Requête: - -``` sql -SELECT if(1, plus(2, 2), plus(2, 6)) -``` - -Résultat: - -``` text -┌─plus(2, 2)─┐ -│ 4 │ -└────────────┘ -``` - -Requête: - -``` sql -SELECT if(0, plus(2, 2), plus(2, 6)) -``` - -Résultat: - -``` text -┌─plus(2, 6)─┐ -│ 8 │ -└────────────┘ -``` - -- `then` et `else` doit avoir le type commun le plus bas. - -**Exemple:** - -Prendre cette `LEFT_RIGHT` table: - -``` sql -SELECT * -FROM LEFT_RIGHT - -┌─left─┬─right─┐ -│ ᴺᵁᴸᴸ │ 4 │ -│ 1 │ 3 │ -│ 2 │ 2 │ -│ 3 │ 1 │ -│ 4 │ ᴺᵁᴸᴸ │ -└──────┴───────┘ -``` - -La requête suivante compare `left` et `right` valeur: - -``` sql -SELECT - left, - right, - if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller -FROM LEFT_RIGHT -WHERE isNotNull(left) AND isNotNull(right) - -┌─left─┬─right─┬─is_smaller──────────────────────────┐ -│ 1 │ 3 │ left is smaller than right │ -│ 2 │ 2 │ right is greater or equal than left │ -│ 3 │ 1 │ right is greater or equal than left │ -└──────┴───────┴─────────────────────────────────────┘ -``` - -Note: `NULL` les valeurs ne sont pas utilisés dans cet exemple, vérifier [Valeurs nulles dans les conditions](#null-values-in-conditionals) section. - -## Opérateur Ternaire {#ternary-operator} - -Il fonctionne même comme `if` fonction. - -Syntaxe: `cond ? then : else` - -Retourner `then` si l' `cond` renvoie la valeur vrai (supérieur à zéro), sinon renvoie `else`. - -- `cond` doit être de type de `UInt8`, et `then` et `else` doit avoir le type commun le plus bas. - -- `then` et `else` peut être `NULL` - -**Voir aussi** - -- [ifNotFinite](other-functions.md#ifnotfinite). - -## multiIf {#multiif} - -Permet d'écrire le [CASE](../operators/index.md#operator_case) opérateur plus compacte dans la requête. - -Syntaxe: `multiIf(cond_1, then_1, cond_2, then_2, ..., else)` - -**Paramètre:** - -- `cond_N` — The condition for the function to return `then_N`. -- `then_N` — The result of the function when executed. -- `else` — The result of the function if none of the conditions is met. - -La fonction accepte `2N+1` paramètre. - -**Valeurs renvoyées** - -La fonction renvoie l'une des valeurs `then_N` ou `else` selon les conditions `cond_N`. - -**Exemple** - -En utilisant à nouveau `LEFT_RIGHT` table. - -``` sql -SELECT - left, - right, - multiIf(left < right, 'left is smaller', left > right, 'left is greater', left = right, 'Both equal', 'Null value') AS result -FROM LEFT_RIGHT - -┌─left─┬─right─┬─result──────────┐ -│ ᴺᵁᴸᴸ │ 4 │ Null value │ -│ 1 │ 3 │ left is smaller │ -│ 2 │ 2 │ Both equal │ -│ 3 │ 1 │ left is greater │ -│ 4 │ ᴺᵁᴸᴸ │ Null value │ -└──────┴───────┴─────────────────┘ -``` - -## Utilisation Directe Des Résultats Conditionnels {#using-conditional-results-directly} - -Les conditions entraînent toujours `0`, `1` ou `NULL`. Vous pouvez donc utiliser des résultats conditionnels directement comme ceci: - -``` sql -SELECT left < right AS is_small -FROM LEFT_RIGHT - -┌─is_small─┐ -│ ᴺᵁᴸᴸ │ -│ 1 │ -│ 0 │ -│ 0 │ -│ ᴺᵁᴸᴸ │ -└──────────┘ -``` - -## Valeurs nulles dans les conditions {#null-values-in-conditionals} - -Lorsque `NULL` les valeurs sont impliqués dans des conditions, le résultat sera également `NULL`. - -``` sql -SELECT - NULL < 1, - 2 < NULL, - NULL < NULL, - NULL = NULL - -┌─less(NULL, 1)─┬─less(2, NULL)─┬─less(NULL, NULL)─┬─equals(NULL, NULL)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└───────────────┴───────────────┴──────────────────┴────────────────────┘ -``` - -Donc, vous devriez construire vos requêtes avec soin si les types sont `Nullable`. - -L'exemple suivant le démontre en omettant d'ajouter la condition égale à `multiIf`. - -``` sql -SELECT - left, - right, - multiIf(left < right, 'left is smaller', left > right, 'right is smaller', 'Both equal') AS faulty_result -FROM LEFT_RIGHT - -┌─left─┬─right─┬─faulty_result────┐ -│ ᴺᵁᴸᴸ │ 4 │ Both equal │ -│ 1 │ 3 │ left is smaller │ -│ 2 │ 2 │ Both equal │ -│ 3 │ 1 │ right is smaller │ -│ 4 │ ᴺᵁᴸᴸ │ Both equal │ -└──────┴───────┴──────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/conditional_functions/) diff --git a/docs/fr/sql-reference/functions/date-time-functions.md b/docs/fr/sql-reference/functions/date-time-functions.md deleted file mode 100644 index d1c16b42d07..00000000000 --- a/docs/fr/sql-reference/functions/date-time-functions.md +++ /dev/null @@ -1,450 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: Travailler avec les Dates et les heures ---- - -# Fonctions pour travailler avec des Dates et des heures {#functions-for-working-with-dates-and-times} - -Support des fuseaux horaires - -Toutes les fonctions pour travailler avec la date et l'heure qui ont une logique d'utilisation pour le fuseau horaire peut accepter un second fuseau horaire argument. Exemple: Asie / Ekaterinbourg. Dans ce cas, ils utilisent le fuseau horaire spécifié au lieu du fuseau horaire local (par défaut). - -``` sql -SELECT - toDateTime('2016-06-15 23:00:00') AS time, - toDate(time) AS date_local, - toDate(time, 'Asia/Yekaterinburg') AS date_yekat, - toString(time, 'US/Samoa') AS time_samoa -``` - -``` text -┌────────────────time─┬─date_local─┬─date_yekat─┬─time_samoa──────────┐ -│ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-16 │ 2016-06-15 09:00:00 │ -└─────────────────────┴────────────┴────────────┴─────────────────────┘ -``` - -Seuls les fuseaux horaires qui diffèrent de L'UTC par un nombre entier d'heures sont pris en charge. - -## toTimeZone {#totimezone} - -Convertir l'heure ou la date et de l'heure au fuseau horaire spécifié. - -## toYear {#toyear} - -Convertit une date ou une date avec l'heure en un numéro UInt16 contenant le numéro d'année (AD). - -## toQuarter {#toquarter} - -Convertit une date ou une date avec l'heure en un numéro UInt8 contenant le numéro de trimestre. - -## toMonth {#tomonth} - -Convertit une date ou une date avec l'heure en un numéro UInt8 contenant le numéro de mois (1-12). - -## toDayOfYear {#todayofyear} - -Convertit une date ou une date avec l'heure en un numéro UInt16 contenant le numéro du jour de l'année (1-366). - -## toDayOfMonth {#todayofmonth} - -Convertit une date ou une date avec le temps à un UInt8 contenant le numéro du jour du mois (1-31). - -## toDayOfWeek {#todayofweek} - -Convertit une date ou une date avec l'heure en un numéro UInt8 contenant le numéro du jour de la semaine (lundi est 1, et dimanche est 7). - -## toHour {#tohour} - -Convertit une date avec l'heure en un nombre UInt8 contenant le numéro de l'heure dans l'Heure de 24 heures (0-23). -This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). - -## toMinute {#tominute} - -Convertit une date avec l'heure en un numéro UInt8 contenant le numéro de la minute de l'heure (0-59). - -## toseconde {#tosecond} - -Convertit une date avec l'heure en un nombre UInt8 contenant le numéro de la seconde dans la minute (0-59). -Les secondes intercalaires ne sont pas comptabilisés. - -## toUnixTimestamp {#to-unix-timestamp} - -Pour L'argument DateTime: convertit la valeur en sa représentation numérique interne (horodatage Unix). -For String argument: analyse datetime from string en fonction du fuseau horaire (second argument optionnel, le fuseau horaire du serveur est utilisé par défaut) et renvoie l'horodatage unix correspondant. -Pour L'argument Date: le comportement n'est pas spécifié. - -**Syntaxe** - -``` sql -toUnixTimestamp(datetime) -toUnixTimestamp(str, [timezone]) -``` - -**Valeur renvoyée** - -- Renvoie l'horodatage unix. - -Type: `UInt32`. - -**Exemple** - -Requête: - -``` sql -SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp -``` - -Résultat: - -``` text -┌─unix_timestamp─┐ -│ 1509836867 │ -└────────────────┘ -``` - -## toStartOfYear {#tostartofyear} - -Arrondit une date ou une date avec l'heure jusqu'au premier jour de l'année. -Renvoie la date. - -## toStartOfISOYear {#tostartofisoyear} - -Arrondit une date ou une date avec l'heure jusqu'au premier jour de L'année ISO. -Renvoie la date. - -## toStartOfQuarter {#tostartofquarter} - -Arrondit une date ou une date avec l'heure jusqu'au premier jour du trimestre. -Le premier jour du trimestre, soit le 1er janvier, 1er avril, 1er juillet ou 1er octobre. -Renvoie la date. - -## toStartOfMonth {#tostartofmonth} - -Arrondit une date ou une date avec l'heure jusqu'au premier jour du mois. -Renvoie la date. - -!!! attention "Attention" - Le comportement de l'analyse des dates incorrectes est spécifique à l'implémentation. ClickHouse peut renvoyer la date zéro, lancer une exception ou faire “natural” débordement. - -## toMonday {#tomonday} - -Arrondit une date ou une date avec l'heure au lundi le plus proche. -Renvoie la date. - -## toStartOfWeek (t \[, mode\]) {#tostartofweektmode} - -Arrondit une date ou une date avec l'heure au dimanche ou au lundi le plus proche par mode. -Renvoie la date. -L'argument mode fonctionne exactement comme l'argument mode de toWeek(). Pour la syntaxe à argument unique, une valeur de mode de 0 est utilisée. - -## toStartOfDay {#tostartofday} - -Arrondit une date avec le temps au début de la journée. - -## toStartOfHour {#tostartofhour} - -Arrondit une date avec le temps au début de l " heure. - -## toStartOfMinute {#tostartofminute} - -Arrondit une date avec le temps au début de la minute. - -## toStartOfFiveMinute {#tostartoffiveminute} - -Arrondit à une date avec l'heure de début de l'intervalle de cinq minutes. - -## toStartOfTenMinutes {#tostartoftenminutes} - -Arrondit une date avec le temps au début de l " intervalle de dix minutes. - -## toStartOfFifteenMinutes {#tostartoffifteenminutes} - -Arrondit la date avec le temps jusqu'au début de l'intervalle de quinze minutes. - -## toStartOfInterval(time_or_data, intervalle x Unité \[, time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} - -Ceci est une généralisation d'autres fonctions nommées `toStartOf*`. Exemple, -`toStartOfInterval(t, INTERVAL 1 year)` renvoie la même chose que `toStartOfYear(t)`, -`toStartOfInterval(t, INTERVAL 1 month)` renvoie la même chose que `toStartOfMonth(t)`, -`toStartOfInterval(t, INTERVAL 1 day)` renvoie la même chose que `toStartOfDay(t)`, -`toStartOfInterval(t, INTERVAL 15 minute)` renvoie la même chose que `toStartOfFifteenMinutes(t)` etc. - -## toTime {#totime} - -Convertit une date avec l'heure en une certaine date fixe, tout en préservant l'heure. - -## toRelativeYearNum {#torelativeyearnum} - -Convertit une date avec l'heure ou la date, le numéro de l'année, à partir d'un certain point fixe dans le passé. - -## toRelativeQuarterNum {#torelativequarternum} - -Convertit une date avec l'heure ou la date au numéro du trimestre, à partir d'un certain point fixe dans le passé. - -## toRelativeMonthNum {#torelativemonthnum} - -Convertit une date avec l'heure ou la date au numéro du mois, à partir d'un certain point fixe dans le passé. - -## toRelativeWeekNum {#torelativeweeknum} - -Convertit une date avec l'heure ou la date, le numéro de la semaine, à partir d'un certain point fixe dans le passé. - -## toRelativeDayNum {#torelativedaynum} - -Convertit une date avec l'heure ou la date au numéro du jour, à partir d'un certain point fixe dans le passé. - -## toRelativeHourNum {#torelativehournum} - -Convertit une date avec l'heure ou la date au nombre de l'heure, à partir d'un certain point fixe dans le passé. - -## toRelativeMinuteNum {#torelativeminutenum} - -Convertit une date avec l'heure ou la date au numéro de la minute, à partir d'un certain point fixe dans le passé. - -## toRelativeSecondNum {#torelativesecondnum} - -Convertit une date avec l'heure ou la date au numéro de la seconde, à partir d'un certain point fixe dans le passé. - -## toISOYear {#toisoyear} - -Convertit une date ou une date avec l'heure en un numéro UInt16 contenant le numéro D'année ISO. - -## toISOWeek {#toisoweek} - -Convertit une date ou une date avec l'heure en un numéro UInt8 contenant le numéro de semaine ISO. - -## toWeek (date \[, mode\]) {#toweekdatemode} - -Cette fonction renvoie le numéro de semaine pour date ou datetime. La forme à deux arguments de toWeek() vous permet de spécifier si la semaine commence le dimanche ou le lundi et si la valeur de retour doit être comprise entre 0 et 53 ou entre 1 et 53. Si l'argument mode est omis, le mode par défaut est 0. -`toISOWeek()`est une fonction de compatibilité équivalente à `toWeek(date,3)`. -Le tableau suivant décrit le fonctionnement de l'argument mode. - -| Mode | Premier jour de la semaine | Gamme | Week 1 is the first week … | -|------|----------------------------|-------|----------------------------------| -| 0 | Dimanche | 0-53 | avec un dimanche cette année | -| 1 | Lundi | 0-53 | avec 4 jours ou plus cette année | -| 2 | Dimanche | 1-53 | avec un dimanche cette année | -| 3 | Lundi | 1-53 | avec 4 jours ou plus cette année | -| 4 | Dimanche | 0-53 | avec 4 jours ou plus cette année | -| 5 | Lundi | 0-53 | avec un lundi cette année | -| 6 | Dimanche | 1-53 | avec 4 jours ou plus cette année | -| 7 | Lundi | 1-53 | avec un lundi cette année | -| 8 | Dimanche | 1-53 | contient Janvier 1 | -| 9 | Lundi | 1-53 | contient Janvier 1 | - -Pour les valeurs de mode avec une signification de “with 4 or more days this year,” les semaines sont numérotées selon ISO 8601: 1988: - -- Si la semaine contenant Janvier 1 A 4 jours ou plus dans la nouvelle année, il est Semaine 1. - -- Sinon, c'est la dernière semaine de l'année précédente, et la semaine prochaine est la semaine 1. - -Pour les valeurs de mode avec une signification de “contains January 1”, la semaine contient Janvier 1 est Semaine 1. Peu importe combien de jours dans la nouvelle année la semaine contenait, même si elle contenait seulement un jour. - -``` sql -toWeek(date, [, mode][, Timezone]) -``` - -**Paramètre** - -- `date` – Date or DateTime. -- `mode` – Optional parameter, Range of values is \[0,9\], default is 0. -- `Timezone` – Optional parameter, it behaves like any other conversion function. - -**Exemple** - -``` sql -SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS week1, toWeek(date,9) AS week9; -``` - -``` text -┌───────date─┬─week0─┬─week1─┬─week9─┐ -│ 2016-12-27 │ 52 │ 52 │ 1 │ -└────────────┴───────┴───────┴───────┘ -``` - -## toYearWeek (date \[, mode\]) {#toyearweekdatemode} - -Retourne l'année et la semaine pour une date. L'année dans le résultat peut être différente de l'année dans l'argument date pour la première et la dernière semaine de l'année. - -L'argument mode fonctionne exactement comme l'argument mode de toWeek(). Pour la syntaxe à argument unique, une valeur de mode de 0 est utilisée. - -`toISOYear()`est une fonction de compatibilité équivalente à `intDiv(toYearWeek(date,3),100)`. - -**Exemple** - -``` sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; -``` - -``` text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ -``` - -## maintenant {#now} - -Accepte zéro argument et renvoie l'heure actuelle à l'un des moments de l'exécution de la requête. -Cette fonction renvoie une constante, même si la requête a pris beaucoup de temps à compléter. - -## aujourd' {#today} - -Accepte zéro argument et renvoie la date actuelle à l'un des moments de l'exécution de la requête. -Le même que ‘toDate(now())’. - -## hier {#yesterday} - -Accepte zéro argument et renvoie la date d'hier à l'un des moments de l'exécution de la requête. -Le même que ‘today() - 1’. - -## l'horaire de diffusion {#timeslot} - -Arrondit le temps à la demi-heure. -Cette fonction est spécifique à Yandex.Metrica, car une demi-heure est le temps minimum pour diviser une session en deux sessions si une balise de suivi affiche les pages vues consécutives d'un seul utilisateur qui diffèrent dans le temps de strictement plus que ce montant. Cela signifie que les tuples (l'ID de balise, l'ID utilisateur et l'intervalle de temps) peuvent être utilisés pour rechercher les pages vues incluses dans la session correspondante. - -## toYYYYMM {#toyyyymm} - -Convertit une date ou une date avec l'heure en un numéro UInt32 contenant le numéro d'année et de mois (AAAA \* 100 + MM). - -## toYYYYMMDD {#toyyyymmdd} - -Convertit une date ou une date avec l'heure en un numéro UInt32 contenant le numéro d'année et de mois (AAAA \* 10000 + MM \* 100 + JJ). - -## toYYYYMMDDhhmmss {#toyyyymmddhhmmss} - -Convertit une date ou une date avec l'heure en un numéro UInt64 contenant le numéro d'année et de mois (AAAA \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). - -## addYears, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addQuarters {#addyears-addmonths-addweeks-adddays-addhours-addminutes-addseconds-addquarters} - -Fonction ajoute une date / DateTime intervalle à une Date / DateTime, puis retourner la Date / DateTime. Exemple: - -``` sql -WITH - toDate('2018-01-01') AS date, - toDateTime('2018-01-01 00:00:00') AS date_time -SELECT - addYears(date, 1) AS add_years_with_date, - addYears(date_time, 1) AS add_years_with_date_time -``` - -``` text -┌─add_years_with_date─┬─add_years_with_date_time─┐ -│ 2019-01-01 │ 2019-01-01 00:00:00 │ -└─────────────────────┴──────────────────────────┘ -``` - -## subtractYears, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractQuarters {#subtractyears-subtractmonths-subtractweeks-subtractdays-subtracthours-subtractminutes-subtractseconds-subtractquarters} - -Fonction soustrayez un intervalle de Date / DateTime à une Date / DateTime, puis renvoyez la Date / DateTime. Exemple: - -``` sql -WITH - toDate('2019-01-01') AS date, - toDateTime('2019-01-01 00:00:00') AS date_time -SELECT - subtractYears(date, 1) AS subtract_years_with_date, - subtractYears(date_time, 1) AS subtract_years_with_date_time -``` - -``` text -┌─subtract_years_with_date─┬─subtract_years_with_date_time─┐ -│ 2018-01-01 │ 2018-01-01 00:00:00 │ -└──────────────────────────┴───────────────────────────────┘ -``` - -## dateDiff {#datediff} - -Renvoie la différence entre deux valeurs Date ou DateTime. - -**Syntaxe** - -``` sql -dateDiff('unit', startdate, enddate, [timezone]) -``` - -**Paramètre** - -- `unit` — Time unit, in which the returned value is expressed. [Chaîne](../syntax.md#syntax-string-literal). - - Supported values: - - | unit | - | ---- | - |second | - |minute | - |hour | - |day | - |week | - |month | - |quarter | - |year | - -- `startdate` — The first time value to compare. [Date](../../sql-reference/data-types/date.md) ou [DateTime](../../sql-reference/data-types/datetime.md). - -- `enddate` — The second time value to compare. [Date](../../sql-reference/data-types/date.md) ou [DateTime](../../sql-reference/data-types/datetime.md). - -- `timezone` — Optional parameter. If specified, it is applied to both `startdate` et `enddate`. Si non spécifié, fuseaux horaires de l' `startdate` et `enddate` sont utilisés. Si elles ne sont pas identiques, le résultat n'est pas spécifié. - -**Valeur renvoyée** - -Différence entre `startdate` et `enddate` exprimé en `unit`. - -Type: `int`. - -**Exemple** - -Requête: - -``` sql -SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00')); -``` - -Résultat: - -``` text -┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐ -│ 25 │ -└────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## intervalle de temps (StartTime, Duration, \[, Size\]) {#timeslotsstarttime-duration-size} - -Pour un intervalle de temps commençant à ‘StartTime’ et de poursuivre pour ‘Duration’ secondes, il renvoie un tableau de moments dans le temps, composé de points de cet intervalle arrondis vers le bas à la ‘Size’ en quelques secondes. ‘Size’ est un paramètre optionnel: une constante UInt32, définie sur 1800 par défaut. -Exemple, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. -Ceci est nécessaire pour rechercher les pages vues dans la session correspondante. - -## formatDateTime(Heure, Format \[, fuseau horaire\]) {#formatdatetime} - -Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. - -Modificateurs pris en charge pour le Format: -(“Example” colonne affiche le résultat de formatage pour le temps `2018-01-02 22:33:44`) - -| Modificateur | Description | Exemple | -|--------------|------------------------------------------------------------------------|------------| -| %C | année divisée par 100 et tronquée en entier (00-99) | 20 | -| %d | jour du mois, zero-rembourré (01-31) | 02 | -| %D | Date courte MM / JJ / AA, équivalente à %m / % d / % y | 01/02/18 | -| % e | jour du mois, rembourré dans l'espace ( 1-31) | 2 | -| %F | date courte AAAA-MM-JJ, équivalente à % Y - % m - % d | 2018-01-02 | -| %H | heure en format 24h (00-23) | 22 | -| %I | heure en format 12h (01-12) | 10 | -| %j | les jours de l'année (001-366) | 002 | -| %m | mois en nombre décimal (01-12) | 01 | -| %M | minute (00-59) | 33 | -| %et | caractère de nouvelle ligne (") | | -| %p | Désignation AM ou PM | PM | -| %R | 24 heures HH:MM temps, équivalent à %H: % M | 22:33 | -| %S | deuxième (00-59) | 44 | -| % t | horizontal-caractère de tabulation (') | | -| %T | Format d'heure ISO 8601 (HH:MM:SS), équivalent à %H: % M:%S | 22:33:44 | -| % u | ISO 8601 jour de la semaine comme numéro avec Lundi comme 1 (1-7) | 2 | -| %V | Numéro de semaine ISO 8601 (01-53) | 01 | -| %W | jour de la semaine comme un nombre décimal avec dimanche comme 0 (0-6) | 2 | -| % y | Année, deux derniers chiffres (00-99) | 18 | -| %Y | An | 2018 | -| %% | signe | % | - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) diff --git a/docs/fr/sql-reference/functions/encoding-functions.md b/docs/fr/sql-reference/functions/encoding-functions.md deleted file mode 100644 index 6c99ed4f32e..00000000000 --- a/docs/fr/sql-reference/functions/encoding-functions.md +++ /dev/null @@ -1,175 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 52 -toc_title: Encodage ---- - -# L'Encodage Des Fonctions {#encoding-functions} - -## char {#char} - -Retourne la chaîne avec la longueur que le nombre d'arguments passés et chaque octet a la valeur de l'argument correspondant. Accepte plusieurs arguments de types numériques. Si la valeur de l'argument est hors de portée du type de données UInt8, elle est convertie en UInt8 avec arrondi et débordement possibles. - -**Syntaxe** - -``` sql -char(number_1, [number_2, ..., number_n]); -``` - -**Paramètre** - -- `number_1, number_2, ..., number_n` — Numerical arguments interpreted as integers. Types: [Int](../../sql-reference/data-types/int-uint.md), [Flottant](../../sql-reference/data-types/float.md). - -**Valeur renvoyée** - -- une chaîne d'octets. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT char(104.1, 101, 108.9, 108.9, 111) AS hello -``` - -Résultat: - -``` text -┌─hello─┐ -│ hello │ -└───────┘ -``` - -Vous pouvez construire une chaîne de codage arbitraire en passant les octets correspondants. Voici un exemple pour UTF-8: - -Requête: - -``` sql -SELECT char(0xD0, 0xBF, 0xD1, 0x80, 0xD0, 0xB8, 0xD0, 0xB2, 0xD0, 0xB5, 0xD1, 0x82) AS hello; -``` - -Résultat: - -``` text -┌─hello──┐ -│ привет │ -└────────┘ -``` - -Requête: - -``` sql -SELECT char(0xE4, 0xBD, 0xA0, 0xE5, 0xA5, 0xBD) AS hello; -``` - -Résultat: - -``` text -┌─hello─┐ -│ 你好 │ -└───────┘ -``` - -## Hex {#hex} - -Renvoie une chaîne contenant la représentation hexadécimale de l'argument. - -**Syntaxe** - -``` sql -hex(arg) -``` - -La fonction utilise des lettres majuscules `A-F` et ne pas utiliser de préfixes (comme `0x`) ou suffixes (comme `h`). - -Pour les arguments entiers, il imprime des chiffres hexadécimaux (“nibbles”) du plus significatif au moins significatif (big endian ou “human readable” ordre). Il commence par l'octet non nul le plus significatif (les octets de début zéro sont omis) mais imprime toujours les deux chiffres de chaque octet même si le chiffre de début est nul. - -Exemple: - -**Exemple** - -Requête: - -``` sql -SELECT hex(1); -``` - -Résultat: - -``` text -01 -``` - -Les valeurs de type `Date` et `DateTime` sont formatés comme des entiers correspondants (le nombre de jours depuis Epoch pour Date et la valeur de L'horodatage Unix pour DateTime). - -Pour `String` et `FixedString`, tous les octets sont simplement codés en deux nombres hexadécimaux. Zéro octets ne sont pas omis. - -Les valeurs des types virgule flottante et décimale sont codées comme leur représentation en mémoire. Comme nous soutenons l'architecture little endian, ils sont codés dans little endian. Zéro octets de début / fin ne sont pas omis. - -**Paramètre** - -- `arg` — A value to convert to hexadecimal. Types: [Chaîne](../../sql-reference/data-types/string.md), [UInt](../../sql-reference/data-types/int-uint.md), [Flottant](../../sql-reference/data-types/float.md), [Décimal](../../sql-reference/data-types/decimal.md), [Date](../../sql-reference/data-types/date.md) ou [DateTime](../../sql-reference/data-types/datetime.md). - -**Valeur renvoyée** - -- Une chaîne avec la représentation hexadécimale de l'argument. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT hex(toFloat32(number)) as hex_presentation FROM numbers(15, 2); -``` - -Résultat: - -``` text -┌─hex_presentation─┐ -│ 00007041 │ -│ 00008041 │ -└──────────────────┘ -``` - -Requête: - -``` sql -SELECT hex(toFloat64(number)) as hex_presentation FROM numbers(15, 2); -``` - -Résultat: - -``` text -┌─hex_presentation─┐ -│ 0000000000002E40 │ -│ 0000000000003040 │ -└──────────────────┘ -``` - -## unhex (str) {#unhexstr} - -Accepte une chaîne contenant un nombre quelconque de chiffres hexadécimaux, et renvoie une chaîne contenant le correspondant octets. Prend en charge les lettres majuscules et minuscules A-F. Le nombre de chiffres hexadécimaux ne doit pas être pair. S'il est impair, le dernier chiffre est interprété comme la moitié la moins significative de l'octet 00-0F. Si la chaîne d'argument contient autre chose que des chiffres hexadécimaux, un résultat défini par l'implémentation est renvoyé (une exception n'est pas levée). -Si vous voulez convertir le résultat en un nombre, vous pouvez utiliser le ‘reverse’ et ‘reinterpretAsType’ fonction. - -## UUIDStringToNum (str) {#uuidstringtonumstr} - -Accepte une chaîne contenant 36 caractères dans le format `123e4567-e89b-12d3-a456-426655440000`, et le renvoie comme un ensemble d'octets dans un FixedString (16). - -## UUIDNumToString (str) {#uuidnumtostringstr} - -Accepte une valeur FixedString (16). Renvoie une chaîne contenant 36 caractères au format texte. - -## bitmaskToList(num) {#bitmasktolistnum} - -Accepte un entier. Renvoie une chaîne contenant la liste des puissances de deux qui totalisent le nombre source lorsqu'il est additionné. Ils sont séparés par des virgules sans espaces au format texte, dans l'ordre croissant. - -## bitmaskToArray(num) {#bitmasktoarraynum} - -Accepte un entier. Renvoie un tableau de nombres UInt64 contenant la liste des puissances de deux qui totalisent le nombre source lorsqu'il est additionné. Les numéros dans le tableau sont dans l'ordre croissant. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/encoding_functions/) diff --git a/docs/fr/sql-reference/functions/ext-dict-functions.md b/docs/fr/sql-reference/functions/ext-dict-functions.md deleted file mode 100644 index 1cec307747d..00000000000 --- a/docs/fr/sql-reference/functions/ext-dict-functions.md +++ /dev/null @@ -1,205 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: Travailler avec des dictionnaires externes ---- - -# Fonctions pour travailler avec des dictionnaires externes {#ext_dict_functions} - -Pour plus d'informations sur la connexion et la configuration de dictionnaires externes, voir [Dictionnaires externes](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). - -## dictGet {#dictget} - -Récupère une valeur d'un dictionnaire externe. - -``` sql -dictGet('dict_name', 'attr_name', id_expr) -dictGetOrDefault('dict_name', 'attr_name', id_expr, default_value_expr) -``` - -**Paramètre** - -- `dict_name` — Name of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `attr_name` — Name of the column of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `id_expr` — Key value. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md) ou [Tuple](../../sql-reference/data-types/tuple.md)- tapez la valeur en fonction de la configuration du dictionnaire. -- `default_value_expr` — Value returned if the dictionary doesn't contain a row with the `id_expr` clé. [Expression](../syntax.md#syntax-expressions) renvoyer la valeur dans le type de données configuré pour `attr_name` attribut. - -**Valeur renvoyée** - -- Si ClickHouse analyse l'attribut avec succès dans le [l'attribut type de données](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes), les fonctions renvoient la valeur du dictionnaire de l'attribut qui correspond à `id_expr`. - -- Si il n'y a pas la clé, correspondant à `id_expr` dans le dictionnaire, puis: - - - `dictGet` returns the content of the `` element specified for the attribute in the dictionary configuration. - - `dictGetOrDefault` returns the value passed as the `default_value_expr` parameter. - -ClickHouse lève une exception si elle ne peut pas analyser la valeur de l'attribut ou si la valeur ne correspond pas au type de données d'attribut. - -**Exemple** - -Créer un fichier texte `ext-dict-text.csv` contenant les éléments suivants: - -``` text -1,1 -2,2 -``` - -La première colonne est `id` la deuxième colonne est `c1`. - -Configurer le dictionnaire externe: - -``` xml - - - ext-dict-test - - - /path-to/ext-dict-test.csv - CSV - - - - - - - - id - - - c1 - UInt32 - - - - 0 - - -``` - -Effectuer la requête: - -``` sql -SELECT - dictGetOrDefault('ext-dict-test', 'c1', number + 1, toUInt32(number * 10)) AS val, - toTypeName(val) AS type -FROM system.numbers -LIMIT 3 -``` - -``` text -┌─val─┬─type───┐ -│ 1 │ UInt32 │ -│ 2 │ UInt32 │ -│ 20 │ UInt32 │ -└─────┴────────┘ -``` - -**Voir Aussi** - -- [Dictionnaires Externes](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) - -## dictHas {#dicthas} - -Vérifie si une clé est présente dans un dictionnaire. - -``` sql -dictHas('dict_name', id_expr) -``` - -**Paramètre** - -- `dict_name` — Name of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `id_expr` — Key value. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md)-le type de la valeur. - -**Valeur renvoyée** - -- 0, si il n'y a pas de clé. -- 1, si il y a une clé. - -Type: `UInt8`. - -## dictGetHierarchy {#dictgethierarchy} - -Crée un tableau contenant tous les parents d'une clé dans le [hiérarchique dictionnaire](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md). - -**Syntaxe** - -``` sql -dictGetHierarchy('dict_name', key) -``` - -**Paramètre** - -- `dict_name` — Name of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `key` — Key value. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md)-le type de la valeur. - -**Valeur renvoyée** - -- Les Parents pour la clé. - -Type: [Tableau (UInt64)](../../sql-reference/data-types/array.md). - -## dictisine {#dictisin} - -Vérifie l'ancêtre d'une clé à travers toute la chaîne hiérarchique dans le dictionnaire. - -``` sql -dictIsIn('dict_name', child_id_expr, ancestor_id_expr) -``` - -**Paramètre** - -- `dict_name` — Name of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `child_id_expr` — Key to be checked. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md)-le type de la valeur. -- `ancestor_id_expr` — Alleged ancestor of the `child_id_expr` clé. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md)-le type de la valeur. - -**Valeur renvoyée** - -- 0, si `child_id_expr` n'est pas un enfant de `ancestor_id_expr`. -- 1, si `child_id_expr` est un enfant de `ancestor_id_expr` ou si `child_id_expr` est un `ancestor_id_expr`. - -Type: `UInt8`. - -## D'Autres Fonctions {#ext_dict_functions-other} - -ClickHouse prend en charge des fonctions spécialisées qui convertissent les valeurs d'attribut de dictionnaire en un type de données spécifique, quelle que soit la configuration du dictionnaire. - -Fonction: - -- `dictGetInt8`, `dictGetInt16`, `dictGetInt32`, `dictGetInt64` -- `dictGetUInt8`, `dictGetUInt16`, `dictGetUInt32`, `dictGetUInt64` -- `dictGetFloat32`, `dictGetFloat64` -- `dictGetDate` -- `dictGetDateTime` -- `dictGetUUID` -- `dictGetString` - -Toutes ces fonctions ont le `OrDefault` modification. Exemple, `dictGetDateOrDefault`. - -Syntaxe: - -``` sql -dictGet[Type]('dict_name', 'attr_name', id_expr) -dictGet[Type]OrDefault('dict_name', 'attr_name', id_expr, default_value_expr) -``` - -**Paramètre** - -- `dict_name` — Name of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `attr_name` — Name of the column of the dictionary. [Chaîne littérale](../syntax.md#syntax-string-literal). -- `id_expr` — Key value. [Expression](../syntax.md#syntax-expressions) de retour d'un [UInt64](../../sql-reference/data-types/int-uint.md)-le type de la valeur. -- `default_value_expr` — Value which is returned if the dictionary doesn't contain a row with the `id_expr` clé. [Expression](../syntax.md#syntax-expressions) renvoyer une valeur dans le type de données configuré pour `attr_name` attribut. - -**Valeur renvoyée** - -- Si ClickHouse analyse l'attribut avec succès dans le [l'attribut type de données](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes), les fonctions renvoient la valeur du dictionnaire de l'attribut qui correspond à `id_expr`. - -- Si il n'est pas demandé `id_expr` dans le dictionnaire,: - - - `dictGet[Type]` returns the content of the `` element specified for the attribute in the dictionary configuration. - - `dictGet[Type]OrDefault` returns the value passed as the `default_value_expr` parameter. - -ClickHouse lève une exception si elle ne peut pas analyser la valeur de l'attribut ou si la valeur ne correspond pas au type de données d'attribut. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/ext_dict_functions/) diff --git a/docs/fr/sql-reference/functions/functions-for-nulls.md b/docs/fr/sql-reference/functions/functions-for-nulls.md deleted file mode 100644 index ef7be728ce7..00000000000 --- a/docs/fr/sql-reference/functions/functions-for-nulls.md +++ /dev/null @@ -1,312 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 63 -toc_title: Travailler avec des arguments nullables ---- - -# Fonctions pour travailler avec des agrégats nullables {#functions-for-working-with-nullable-aggregates} - -## isNull {#isnull} - -Vérifie si l'argument est [NULL](../../sql-reference/syntax.md#null-literal). - -``` sql -isNull(x) -``` - -**Paramètre** - -- `x` — A value with a non-compound data type. - -**Valeur renvoyée** - -- `1` si `x` être `NULL`. -- `0` si `x` n'est pas `NULL`. - -**Exemple** - -Table d'entrée - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 3 │ -└───┴──────┘ -``` - -Requête - -``` sql -SELECT x FROM t_null WHERE isNull(y) -``` - -``` text -┌─x─┐ -│ 1 │ -└───┘ -``` - -## isNotNull {#isnotnull} - -Vérifie si l'argument est [NULL](../../sql-reference/syntax.md#null-literal). - -``` sql -isNotNull(x) -``` - -**Paramètre:** - -- `x` — A value with a non-compound data type. - -**Valeur renvoyée** - -- `0` si `x` être `NULL`. -- `1` si `x` n'est pas `NULL`. - -**Exemple** - -Table d'entrée - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 3 │ -└───┴──────┘ -``` - -Requête - -``` sql -SELECT x FROM t_null WHERE isNotNull(y) -``` - -``` text -┌─x─┐ -│ 2 │ -└───┘ -``` - -## fusionner {#coalesce} - -Vérifie de gauche à droite si `NULL` les arguments ont été passés et renvoie le premier non-`NULL` argument. - -``` sql -coalesce(x,...) -``` - -**Paramètre:** - -- N'importe quel nombre de paramètres d'un type non composé. Tous les paramètres doivent être compatibles par type de données. - -**Valeurs renvoyées** - -- Le premier non-`NULL` argument. -- `NULL` si tous les arguments sont `NULL`. - -**Exemple** - -Considérez une liste de contacts qui peuvent spécifier plusieurs façons de contacter un client. - -``` text -┌─name─────┬─mail─┬─phone─────┬──icq─┐ -│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ -│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└──────────┴──────┴───────────┴──────┘ -``` - -Le `mail` et `phone` les champs sont de type Chaîne de caractères, mais la `icq` le terrain est `UInt32`, de sorte qu'il doit être converti en `String`. - -Obtenir la première méthode de contact pour le client à partir de la liste de contacts: - -``` sql -SELECT coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook -``` - -``` text -┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ -│ client 1 │ 123-45-67 │ -│ client 2 │ ᴺᵁᴸᴸ │ -└──────────┴──────────────────────────────────────────────────────┘ -``` - -## ifNull {#ifnull} - -Renvoie une valeur alternative si l'argument principal est `NULL`. - -``` sql -ifNull(x,alt) -``` - -**Paramètre:** - -- `x` — The value to check for `NULL`. -- `alt` — The value that the function returns if `x` être `NULL`. - -**Valeurs renvoyées** - -- Valeur `x`, si `x` n'est pas `NULL`. -- Valeur `alt`, si `x` être `NULL`. - -**Exemple** - -``` sql -SELECT ifNull('a', 'b') -``` - -``` text -┌─ifNull('a', 'b')─┐ -│ a │ -└──────────────────┘ -``` - -``` sql -SELECT ifNull(NULL, 'b') -``` - -``` text -┌─ifNull(NULL, 'b')─┐ -│ b │ -└───────────────────┘ -``` - -## nullIf {#nullif} - -Retourner `NULL` si les arguments sont égaux. - -``` sql -nullIf(x, y) -``` - -**Paramètre:** - -`x`, `y` — Values for comparison. They must be compatible types, or ClickHouse will generate an exception. - -**Valeurs renvoyées** - -- `NULL` si les arguments sont égaux. -- Le `x` valeur, si les arguments ne sont pas égaux. - -**Exemple** - -``` sql -SELECT nullIf(1, 1) -``` - -``` text -┌─nullIf(1, 1)─┐ -│ ᴺᵁᴸᴸ │ -└──────────────┘ -``` - -``` sql -SELECT nullIf(1, 2) -``` - -``` text -┌─nullIf(1, 2)─┐ -│ 1 │ -└──────────────┘ -``` - -## assumeNotNull {#assumenotnull} - -Résultats dans une valeur de type [Nullable](../../sql-reference/data-types/nullable.md) pour un non- `Nullable` si la valeur n'est pas `NULL`. - -``` sql -assumeNotNull(x) -``` - -**Paramètre:** - -- `x` — The original value. - -**Valeurs renvoyées** - -- La valeur d'origine du non-`Nullable` type, si elle n'est pas `NULL`. -- La valeur par défaut pour le non-`Nullable` Tapez si la valeur d'origine était `NULL`. - -**Exemple** - -Envisager l' `t_null` table. - -``` sql -SHOW CREATE TABLE t_null -``` - -``` text -┌─statement─────────────────────────────────────────────────────────────────┐ -│ CREATE TABLE default.t_null ( x Int8, y Nullable(Int8)) ENGINE = TinyLog │ -└───────────────────────────────────────────────────────────────────────────┘ -``` - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 3 │ -└───┴──────┘ -``` - -Appliquer le `assumeNotNull` la fonction de la `y` colonne. - -``` sql -SELECT assumeNotNull(y) FROM t_null -``` - -``` text -┌─assumeNotNull(y)─┐ -│ 0 │ -│ 3 │ -└──────────────────┘ -``` - -``` sql -SELECT toTypeName(assumeNotNull(y)) FROM t_null -``` - -``` text -┌─toTypeName(assumeNotNull(y))─┐ -│ Int8 │ -│ Int8 │ -└──────────────────────────────┘ -``` - -## toNullable {#tonullable} - -Convertit le type d'argument en `Nullable`. - -``` sql -toNullable(x) -``` - -**Paramètre:** - -- `x` — The value of any non-compound type. - -**Valeur renvoyée** - -- La valeur d'entrée avec un `Nullable` type. - -**Exemple** - -``` sql -SELECT toTypeName(10) -``` - -``` text -┌─toTypeName(10)─┐ -│ UInt8 │ -└────────────────┘ -``` - -``` sql -SELECT toTypeName(toNullable(10)) -``` - -``` text -┌─toTypeName(toNullable(10))─┐ -│ Nullable(UInt8) │ -└────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/functions_for_nulls/) diff --git a/docs/fr/sql-reference/functions/geo.md b/docs/fr/sql-reference/functions/geo.md deleted file mode 100644 index a89f03c7216..00000000000 --- a/docs/fr/sql-reference/functions/geo.md +++ /dev/null @@ -1,510 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 62 -toc_title: "Travailler avec des coordonn\xE9es g\xE9ographiques" ---- - -# Fonctions pour travailler avec des coordonnées géographiques {#functions-for-working-with-geographical-coordinates} - -## greatCircleDistance {#greatcircledistance} - -Calculer la distance entre deux points sur la surface de la Terre en utilisant [la formule du grand cercle](https://en.wikipedia.org/wiki/Great-circle_distance). - -``` sql -greatCircleDistance(lon1Deg, lat1Deg, lon2Deg, lat2Deg) -``` - -**Les paramètres d'entrée** - -- `lon1Deg` — Longitude of the first point in degrees. Range: `[-180°, 180°]`. -- `lat1Deg` — Latitude of the first point in degrees. Range: `[-90°, 90°]`. -- `lon2Deg` — Longitude of the second point in degrees. Range: `[-180°, 180°]`. -- `lat2Deg` — Latitude of the second point in degrees. Range: `[-90°, 90°]`. - -Les valeurs positives correspondent à la latitude nord et à la longitude Est, et les valeurs négatives à la latitude Sud et à la longitude ouest. - -**Valeur renvoyée** - -La distance entre deux points sur la surface de la Terre, en mètres. - -Génère une exception lorsque les valeurs des paramètres d'entrée se situent en dehors de la plage. - -**Exemple** - -``` sql -SELECT greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673) -``` - -``` text -┌─greatCircleDistance(55.755831, 37.617673, -55.755831, -37.617673)─┐ -│ 14132374.194975413 │ -└───────────────────────────────────────────────────────────────────┘ -``` - -## pointInEllipses {#pointinellipses} - -Vérifie si le point appartient à au moins une des ellipses. -Coordonnées géométriques sont dans le système de coordonnées Cartésiennes. - -``` sql -pointInEllipses(x, y, x₀, y₀, a₀, b₀,...,xₙ, yₙ, aₙ, bₙ) -``` - -**Les paramètres d'entrée** - -- `x, y` — Coordinates of a point on the plane. -- `xᵢ, yᵢ` — Coordinates of the center of the `i`-ème points de suspension. -- `aᵢ, bᵢ` — Axes of the `i`- e ellipse en unités de coordonnées x, Y. - -Les paramètres d'entrée doivent être `2+4⋅n`, où `n` est le nombre de points de suspension. - -**Valeurs renvoyées** - -`1` si le point est à l'intérieur d'au moins l'un des ellipses; `0`si elle ne l'est pas. - -**Exemple** - -``` sql -SELECT pointInEllipses(10., 10., 10., 9.1, 1., 0.9999) -``` - -``` text -┌─pointInEllipses(10., 10., 10., 9.1, 1., 0.9999)─┐ -│ 1 │ -└─────────────────────────────────────────────────┘ -``` - -## pointtinpolygon {#pointinpolygon} - -Vérifie si le point appartient au polygone sur l'avion. - -``` sql -pointInPolygon((x, y), [(a, b), (c, d) ...], ...) -``` - -**Les valeurs d'entrée** - -- `(x, y)` — Coordinates of a point on the plane. Data type — [Tuple](../../sql-reference/data-types/tuple.md) — A tuple of two numbers. -- `[(a, b), (c, d) ...]` — Polygon vertices. Data type — [Tableau](../../sql-reference/data-types/array.md). Chaque sommet est représenté par une paire de coordonnées `(a, b)`. Les sommets doivent être spécifiés dans le sens horaire ou antihoraire. Le nombre minimum de sommets est 3. Le polygone doit être constante. -- La fonction prend également en charge les polygones avec des trous (découper des sections). Dans ce cas, ajoutez des polygones qui définissent les sections découpées en utilisant des arguments supplémentaires de la fonction. La fonction ne prend pas en charge les polygones non simplement connectés. - -**Valeurs renvoyées** - -`1` si le point est à l'intérieur du polygone, `0` si elle ne l'est pas. -Si le point est sur la limite du polygone, la fonction peut renvoyer 0 ou 1. - -**Exemple** - -``` sql -SELECT pointInPolygon((3., 3.), [(6, 0), (8, 4), (5, 8), (0, 2)]) AS res -``` - -``` text -┌─res─┐ -│ 1 │ -└─────┘ -``` - -## geohashEncode {#geohashencode} - -Encode la latitude et la longitude en tant que chaîne geohash, voir (http://geohash.org/, https://en.wikipedia.org/wiki/Geohash). - -``` sql -geohashEncode(longitude, latitude, [precision]) -``` - -**Les valeurs d'entrée** - -- longitude longitude partie de la coordonnée que vous souhaitez encoder. Flottant dans la gamme`[-180°, 180°]` -- latitude latitude partie de la coordonnée que vous souhaitez encoder. Flottant dans la gamme `[-90°, 90°]` -- precision-facultatif, longueur de la chaîne codée résultante, par défaut `12`. Entier dans la gamme `[1, 12]`. Toute valeur inférieure à `1` ou supérieure à `12` silencieusement converti à `12`. - -**Valeurs renvoyées** - -- alphanumérique `String` de coordonnées codées (la version modifiée de l'alphabet de codage base32 est utilisée). - -**Exemple** - -``` sql -SELECT geohashEncode(-5.60302734375, 42.593994140625, 0) AS res -``` - -``` text -┌─res──────────┐ -│ ezs42d000000 │ -└──────────────┘ -``` - -## geohashDecode {#geohashdecode} - -Décode toute chaîne codée geohash en longitude et latitude. - -**Les valeurs d'entrée** - -- chaîne codée-chaîne codée geohash. - -**Valeurs renvoyées** - -- (longitude, latitude) - 2-n-uplet de `Float64` les valeurs de longitude et de latitude. - -**Exemple** - -``` sql -SELECT geohashDecode('ezs42') AS res -``` - -``` text -┌─res─────────────────────────────┐ -│ (-5.60302734375,42.60498046875) │ -└─────────────────────────────────┘ -``` - -## geoToH3 {#geotoh3} - -Retourner [H3](https://uber.github.io/h3/#/documentation/overview/introduction) point d'indice `(lon, lat)` avec une résolution spécifiée. - -[H3](https://uber.github.io/h3/#/documentation/overview/introduction) est un système d'indexation géographique où la surface de la Terre divisée en carreaux hexagonaux même. Ce système est hiérarchique, c'est-à-dire que chaque hexagone au niveau supérieur peut être divisé en sept, même mais plus petits, etc. - -Cet indice est principalement utilisé pour les emplacements de bucketing et d'autres manipulations géospatiales. - -**Syntaxe** - -``` sql -geoToH3(lon, lat, resolution) -``` - -**Paramètre** - -- `lon` — Longitude. Type: [Float64](../../sql-reference/data-types/float.md). -- `lat` — Latitude. Type: [Float64](../../sql-reference/data-types/float.md). -- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Numéro d'indice hexagonal. -- 0 en cas d'erreur. - -Type: `UInt64`. - -**Exemple** - -Requête: - -``` sql -SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index -``` - -Résultat: - -``` text -┌────────────h3Index─┐ -│ 644325524701193974 │ -└────────────────────┘ -``` - -## geohashesInBox {#geohashesinbox} - -Renvoie un tableau de chaînes codées geohash de précision donnée qui tombent à l'intérieur et croisent les limites d'une boîte donnée, essentiellement une grille 2D aplatie en tableau. - -**Les valeurs d'entrée** - -- longitude_min-longitude min, valeur flottante dans la plage `[-180°, 180°]` -- latitude_min-latitude min, valeur flottante dans la plage `[-90°, 90°]` -- longitude_max-longitude maximale, valeur flottante dans la plage `[-180°, 180°]` -- latitude_max-latitude maximale, valeur flottante dans la plage `[-90°, 90°]` -- précision - geohash précision, `UInt8` dans la gamme `[1, 12]` - -Veuillez noter que tous les paramètres de coordonnées doit être du même type: soit `Float32` ou `Float64`. - -**Valeurs renvoyées** - -- gamme de précision de longues chaînes de geohash-boîtes couvrant la zone, vous ne devriez pas compter sur l'ordre des éléments. -- \[\] - tableau vide si *min* les valeurs de *latitude* et *longitude* ne sont pas moins de correspondant *Max* valeur. - -Veuillez noter que la fonction lancera une exception si le tableau résultant a plus de 10'000'000 éléments. - -**Exemple** - -``` sql -SELECT geohashesInBox(24.48, 40.56, 24.785, 40.81, 4) AS thasos -``` - -``` text -┌─thasos──────────────────────────────────────┐ -│ ['sx1q','sx1r','sx32','sx1w','sx1x','sx38'] │ -└─────────────────────────────────────────────┘ -``` - -## h3GetBaseCell {#h3getbasecell} - -Renvoie le numéro de cellule de base de l'index. - -**Syntaxe** - -``` sql -h3GetBaseCell(index) -``` - -**Paramètre** - -- `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Numéro de cellule de base hexagonale. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3GetBaseCell(612916788725809151) as basecell -``` - -Résultat: - -``` text -┌─basecell─┐ -│ 12 │ -└──────────┘ -``` - -## h3HexAreaM2 {#h3hexaream2} - -Surface hexagonale Moyenne en mètres carrés à la résolution donnée. - -**Syntaxe** - -``` sql -h3HexAreaM2(resolution) -``` - -**Paramètre** - -- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Area in m². Type: [Float64](../../sql-reference/data-types/float.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3HexAreaM2(13) as area -``` - -Résultat: - -``` text -┌─area─┐ -│ 43.9 │ -└──────┘ -``` - -## h3IndexesAreNeighbors {#h3indexesareneighbors} - -Renvoie si les H3Indexes fournis sont voisins ou non. - -**Syntaxe** - -``` sql -h3IndexesAreNeighbors(index1, index2) -``` - -**Paramètre** - -- `index1` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -- `index2` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Retourner `1` si les index sont voisins, `0` autrement. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3IndexesAreNeighbors(617420388351344639, 617420388352655359) AS n -``` - -Résultat: - -``` text -┌─n─┐ -│ 1 │ -└───┘ -``` - -## h3enfants {#h3tochildren} - -Retourne un tableau avec les index enfants de l'index donné. - -**Syntaxe** - -``` sql -h3ToChildren(index, resolution) -``` - -**Paramètre** - -- `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Tableau avec les index H3 enfants. Tableau de type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3ToChildren(599405990164561919, 6) AS children -``` - -Résultat: - -``` text -┌─children───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ [603909588852408319,603909588986626047,603909589120843775,603909589255061503,603909589389279231,603909589523496959,603909589657714687] │ -└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## h3ToParent {#h3toparent} - -Renvoie l'index parent (plus grossier) contenant l'index donné. - -**Syntaxe** - -``` sql -h3ToParent(index, resolution) -``` - -**Paramètre** - -- `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -- `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Parent H3 index. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3ToParent(599405990164561919, 3) as parent -``` - -Résultat: - -``` text -┌─────────────parent─┐ -│ 590398848891879423 │ -└────────────────────┘ -``` - -## h3ToString {#h3tostring} - -Convertit la représentation H3Index de l'index en représentation de chaîne. - -``` sql -h3ToString(index) -``` - -**Paramètre** - -- `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- Représentation en chaîne de l'index H3. Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3ToString(617420388352917503) as h3_string -``` - -Résultat: - -``` text -┌─h3_string───────┐ -│ 89184926cdbffff │ -└─────────────────┘ -``` - -## stringToH3 {#stringtoh3} - -Convertit la représentation de chaîne en représentation H3Index (UInt64). - -``` sql -stringToH3(index_str) -``` - -**Paramètre** - -- `index_str` — String representation of the H3 index. Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Valeurs renvoyées** - -- Numéro d'indice hexagonal. Renvoie 0 en cas d'erreur. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT stringToH3('89184926cc3ffff') as index -``` - -Résultat: - -``` text -┌──────────────index─┐ -│ 617420388351344639 │ -└────────────────────┘ -``` - -## h3grésolution {#h3getresolution} - -Retourne la résolution de l'index. - -**Syntaxe** - -``` sql -h3GetResolution(index) -``` - -**Paramètre** - -- `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Valeurs renvoyées** - -- L'indice de la résolution. Gamme: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT h3GetResolution(617420388352917503) as res -``` - -Résultat: - -``` text -┌─res─┐ -│ 9 │ -└─────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/geo/) diff --git a/docs/fr/sql-reference/functions/hash-functions.md b/docs/fr/sql-reference/functions/hash-functions.md deleted file mode 100644 index 3b0f92dd4f8..00000000000 --- a/docs/fr/sql-reference/functions/hash-functions.md +++ /dev/null @@ -1,484 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 50 -toc_title: Hachage ---- - -# Les Fonctions De Hachage {#hash-functions} - -Les fonctions de hachage peuvent être utilisées pour le brassage pseudo-aléatoire déterministe des éléments. - -## halfMD5 {#hash-functions-halfmd5} - -[Interpréter](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) tous les paramètres d'entrée sous forme de chaînes et calcule le [MD5](https://en.wikipedia.org/wiki/MD5) la valeur de hachage pour chacun d'eux. Puis combine les hachages, prend les 8 premiers octets du hachage de la chaîne résultante, et les interprète comme `UInt64` dans l'ordre des octets big-endian. - -``` sql -halfMD5(par1, ...) -``` - -La fonction est relativement lente (5 millions de chaînes courtes par seconde par cœur de processeur). -Envisager l'utilisation de la [sipHash64](#hash_functions-siphash64) la fonction la place. - -**Paramètre** - -La fonction prend un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -A [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT halfMD5(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS halfMD5hash, toTypeName(halfMD5hash) AS type -``` - -``` text -┌────────halfMD5hash─┬─type───┐ -│ 186182704141653334 │ UInt64 │ -└────────────────────┴────────┘ -``` - -## MD5 {#hash_functions-md5} - -Calcule le MD5 à partir d'une chaîne et renvoie L'ensemble d'octets résultant en tant que FixedString(16). -Si vous n'avez pas besoin de MD5 en particulier, mais que vous avez besoin d'un hachage cryptographique 128 bits décent, utilisez le ‘sipHash128’ la fonction la place. -Si vous voulez obtenir le même résultat que la sortie de l'utilitaire md5sum, utilisez lower (hex(MD5 (s))). - -## sipHash64 {#hash_functions-siphash64} - -Produit un 64 bits [SipHash](https://131002.net/siphash/) la valeur de hachage. - -``` sql -sipHash64(par1,...) -``` - -C'est une fonction de hachage cryptographique. Il fonctionne au moins trois fois plus vite que le [MD5](#hash_functions-md5) fonction. - -Fonction [interpréter](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) tous les paramètres d'entrée sous forme de chaînes et calcule la valeur de hachage pour chacun d'eux. Puis combine les hachages par l'algorithme suivant: - -1. Après avoir haché tous les paramètres d'entrée, la fonction obtient le tableau de hachages. -2. La fonction prend le premier et le second éléments et calcule un hachage pour le tableau d'entre eux. -3. Ensuite, la fonction prend la valeur de hachage, calculée à l'étape précédente, et le troisième élément du tableau de hachage initial, et calcule un hachage pour le tableau d'entre eux. -4. L'étape précédente est répétée pour tous les éléments restants de la période initiale de hachage tableau. - -**Paramètre** - -La fonction prend un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -A [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT sipHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS SipHash, toTypeName(SipHash) AS type -``` - -``` text -┌──────────────SipHash─┬─type───┐ -│ 13726873534472839665 │ UInt64 │ -└──────────────────────┴────────┘ -``` - -## sipHash128 {#hash_functions-siphash128} - -Calcule SipHash à partir d'une chaîne. -Accepte un argument de type chaîne. Renvoie FixedString (16). -Diffère de sipHash64 en ce que l'état de pliage xor final n'est effectué que jusqu'à 128 bits. - -## cityHash64 {#cityhash64} - -Produit un 64 bits [CityHash](https://github.com/google/cityhash) la valeur de hachage. - -``` sql -cityHash64(par1,...) -``` - -Ceci est une fonction de hachage non cryptographique rapide. Il utilise L'algorithme CityHash pour les paramètres de chaîne et la fonction de hachage rapide non cryptographique spécifique à l'implémentation pour les paramètres avec d'autres types de données. La fonction utilise le combinateur CityHash pour obtenir les résultats finaux. - -**Paramètre** - -La fonction prend un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -A [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -Appelez exemple: - -``` sql -SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS CityHash, toTypeName(CityHash) AS type -``` - -``` text -┌─────────────CityHash─┬─type───┐ -│ 12072650598913549138 │ UInt64 │ -└──────────────────────┴────────┘ -``` - -L'exemple suivant montre comment calculer la somme de l'ensemble de la table avec précision jusqu'à la ligne de commande: - -``` sql -SELECT groupBitXor(cityHash64(*)) FROM table -``` - -## intHash32 {#inthash32} - -Calcule un code de hachage 32 bits à partir de n'importe quel type d'entier. -C'est une fonction de hachage non cryptographique relativement rapide de qualité moyenne pour les nombres. - -## intHash64 {#inthash64} - -Calcule un code de hachage 64 bits à partir de n'importe quel type d'entier. -Il fonctionne plus vite que intHash32. Qualité moyenne. - -## SHA1 {#sha1} - -## SHA224 {#sha224} - -## SHA256 {#sha256} - -Calcule SHA-1, SHA-224 ou SHA-256 à partir d'une chaîne et renvoie l'ensemble d'octets résultant en tant que FixedString(20), FixedString(28) ou FixedString(32). -La fonction fonctionne assez lentement (SHA-1 traite environ 5 millions de chaînes courtes par seconde par cœur de processeur, tandis que SHA-224 et SHA-256 traitent environ 2,2 millions). -Nous vous recommandons d'utiliser cette fonction uniquement dans les cas où vous avez besoin d'une fonction de hachage spécifique et que vous ne pouvez pas la sélectionner. -Même dans ces cas, nous vous recommandons d'appliquer la fonction hors ligne et de pré-calculer les valeurs lors de leur insertion dans la table, au lieu de l'appliquer dans SELECTS. - -## URLHash(url \[, N\]) {#urlhashurl-n} - -Une fonction de hachage non cryptographique rapide et de qualité décente pour une chaîne obtenue à partir d'une URL en utilisant un type de normalisation. -`URLHash(s)` – Calculates a hash from a string without one of the trailing symbols `/`,`?` ou `#` à la fin, si elle est présente. -`URLHash(s, N)` – Calculates a hash from a string up to the N level in the URL hierarchy, without one of the trailing symbols `/`,`?` ou `#` à la fin, si elle est présente. -Les niveaux sont les mêmes que dans URLHierarchy. Cette fonction est spécifique à Yandex.Metrica. - -## farmHash64 {#farmhash64} - -Produit un 64 bits [FarmHash](https://github.com/google/farmhash) la valeur de hachage. - -``` sql -farmHash64(par1, ...) -``` - -La fonction utilise le `Hash64` la méthode de tous les [les méthodes disponibles](https://github.com/google/farmhash/blob/master/src/farmhash.h). - -**Paramètre** - -La fonction prend un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -A [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT farmHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS FarmHash, toTypeName(FarmHash) AS type -``` - -``` text -┌─────────────FarmHash─┬─type───┐ -│ 17790458267262532859 │ UInt64 │ -└──────────────────────┴────────┘ -``` - -## javaHash {#hash_functions-javahash} - -Calculer [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) à partir d'une chaîne. Cette fonction de hachage n'est ni rapide ni de bonne qualité. La seule raison de l'utiliser est lorsque cet algorithme est déjà utilisé dans un autre système et que vous devez calculer exactement le même résultat. - -**Syntaxe** - -``` sql -SELECT javaHash(''); -``` - -**Valeur renvoyée** - -A `Int32` valeur de hachage du type de données. - -**Exemple** - -Requête: - -``` sql -SELECT javaHash('Hello, world!'); -``` - -Résultat: - -``` text -┌─javaHash('Hello, world!')─┐ -│ -1880044555 │ -└───────────────────────────┘ -``` - -## javaHashUTF16LE {#javahashutf16le} - -Calculer [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) à partir d'une chaîne, en supposant qu'elle contient des octets représentant une chaîne en encodage UTF-16LE. - -**Syntaxe** - -``` sql -javaHashUTF16LE(stringUtf16le) -``` - -**Paramètre** - -- `stringUtf16le` — a string in UTF-16LE encoding. - -**Valeur renvoyée** - -A `Int32` valeur de hachage du type de données. - -**Exemple** - -Requête correcte avec une chaîne codée UTF-16LE. - -Requête: - -``` sql -SELECT javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le')) -``` - -Résultat: - -``` text -┌─javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le'))─┐ -│ 3556498 │ -└──────────────────────────────────────────────────────────────┘ -``` - -## hiveHash {#hash-functions-hivehash} - -Calculer `HiveHash` à partir d'une chaîne. - -``` sql -SELECT hiveHash(''); -``` - -C'est juste [JavaHash](#hash_functions-javahash) avec le bit de signe mis à zéro. Cette fonction est utilisée dans [Apache Hive](https://en.wikipedia.org/wiki/Apache_Hive) pour les versions antérieures à la version 3.0. Cette fonction de hachage n'est ni rapide ni de bonne qualité. La seule raison de l'utiliser est lorsque cet algorithme est déjà utilisé dans un autre système et que vous devez calculer exactement le même résultat. - -**Valeur renvoyée** - -A `Int32` valeur de hachage du type de données. - -Type: `hiveHash`. - -**Exemple** - -Requête: - -``` sql -SELECT hiveHash('Hello, world!'); -``` - -Résultat: - -``` text -┌─hiveHash('Hello, world!')─┐ -│ 267439093 │ -└───────────────────────────┘ -``` - -## metroHash64 {#metrohash64} - -Produit un 64 bits [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/) la valeur de hachage. - -``` sql -metroHash64(par1, ...) -``` - -**Paramètre** - -La fonction prend un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -A [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT metroHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MetroHash, toTypeName(MetroHash) AS type -``` - -``` text -┌────────────MetroHash─┬─type───┐ -│ 14235658766382344533 │ UInt64 │ -└──────────────────────┴────────┘ -``` - -## jumpConsistentHash {#jumpconsistenthash} - -Calcule JumpConsistentHash forme un UInt64. -Accepte deux arguments: une clé de type UInt64 et le nombre de compartiments. Renvoie Int32. -Pour plus d'informations, voir le lien: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) - -## murmurHash2_32, murmurHash2_64 {#murmurhash2-32-murmurhash2-64} - -Produit un [MurmurHash2](https://github.com/aappleby/smhasher) la valeur de hachage. - -``` sql -murmurHash2_32(par1, ...) -murmurHash2_64(par1, ...) -``` - -**Paramètre** - -Les deux fonctions prennent un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -- Le `murmurHash2_32` fonction renvoie la valeur de hachage ayant le [UInt32](../../sql-reference/data-types/int-uint.md) type de données. -- Le `murmurHash2_64` fonction renvoie la valeur de hachage ayant le [UInt64](../../sql-reference/data-types/int-uint.md) type de données. - -**Exemple** - -``` sql -SELECT murmurHash2_64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MurmurHash2, toTypeName(MurmurHash2) AS type -``` - -``` text -┌──────────MurmurHash2─┬─type───┐ -│ 11832096901709403633 │ UInt64 │ -└──────────────────────┴────────┘ -``` - -## gccMurmurHash {#gccmurmurhash} - -Calcule un 64 bits [MurmurHash2](https://github.com/aappleby/smhasher) valeur de hachage utilisant la même graine de hachage que [gcc](https://github.com/gcc-mirror/gcc/blob/41d6b10e96a1de98e90a7c0378437c3255814b16/libstdc%2B%2B-v3/include/bits/functional_hash.h#L191). Il est portable entre Clang et GCC construit. - -**Syntaxe** - -``` sql -gccMurmurHash(par1, ...); -``` - -**Paramètre** - -- `par1, ...` — A variable number of parameters that can be any of the [types de données pris en charge](../../sql-reference/data-types/index.md#data_types). - -**Valeur renvoyée** - -- Valeur de hachage calculée. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT - gccMurmurHash(1, 2, 3) AS res1, - gccMurmurHash(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))) AS res2 -``` - -Résultat: - -``` text -┌─────────────────res1─┬────────────────res2─┐ -│ 12384823029245979431 │ 1188926775431157506 │ -└──────────────────────┴─────────────────────┘ -``` - -## murmurHash3_32, murmurHash3_64 {#murmurhash3-32-murmurhash3-64} - -Produit un [MurmurHash3](https://github.com/aappleby/smhasher) la valeur de hachage. - -``` sql -murmurHash3_32(par1, ...) -murmurHash3_64(par1, ...) -``` - -**Paramètre** - -Les deux fonctions prennent un nombre variable de paramètres d'entrée. Les paramètres peuvent être tout de la [types de données pris en charge](../../sql-reference/data-types/index.md). - -**Valeur Renvoyée** - -- Le `murmurHash3_32` la fonction retourne un [UInt32](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. -- Le `murmurHash3_64` la fonction retourne un [UInt64](../../sql-reference/data-types/int-uint.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MurmurHash3, toTypeName(MurmurHash3) AS type -``` - -``` text -┌─MurmurHash3─┬─type───┐ -│ 2152717 │ UInt32 │ -└─────────────┴────────┘ -``` - -## murmurHash3_128 {#murmurhash3-128} - -Produit de 128 bits [MurmurHash3](https://github.com/aappleby/smhasher) la valeur de hachage. - -``` sql -murmurHash3_128( expr ) -``` - -**Paramètre** - -- `expr` — [Expression](../syntax.md#syntax-expressions) de retour d'un [Chaîne](../../sql-reference/data-types/string.md)-le type de la valeur. - -**Valeur Renvoyée** - -A [FixedString (16)](../../sql-reference/data-types/fixedstring.md) valeur de hachage du type de données. - -**Exemple** - -``` sql -SELECT murmurHash3_128('example_string') AS MurmurHash3, toTypeName(MurmurHash3) AS type -``` - -``` text -┌─MurmurHash3──────┬─type────────────┐ -│ 6�1�4"S5KT�~~q │ FixedString(16) │ -└──────────────────┴─────────────────┘ -``` - -## xxHash32, xxHash64 {#hash-functions-xxhash32} - -Calculer `xxHash` à partir d'une chaîne. Il est proposé en deux saveurs, 32 et 64 bits. - -``` sql -SELECT xxHash32(''); - -OR - -SELECT xxHash64(''); -``` - -**Valeur renvoyée** - -A `Uint32` ou `Uint64` valeur de hachage du type de données. - -Type: `xxHash`. - -**Exemple** - -Requête: - -``` sql -SELECT xxHash32('Hello, world!'); -``` - -Résultat: - -``` text -┌─xxHash32('Hello, world!')─┐ -│ 834093149 │ -└───────────────────────────┘ -``` - -**Voir Aussi** - -- [xxHash](http://cyan4973.github.io/xxHash/). - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/hash_functions/) diff --git a/docs/fr/sql-reference/functions/higher-order-functions.md b/docs/fr/sql-reference/functions/higher-order-functions.md deleted file mode 100644 index ac24b67bb97..00000000000 --- a/docs/fr/sql-reference/functions/higher-order-functions.md +++ /dev/null @@ -1,264 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 57 -toc_title: "D'Ordre Sup\xE9rieur" ---- - -# Fonctions d'ordre supérieur {#higher-order-functions} - -## `->` opérateur, fonction lambda (params, expr) {#operator-lambdaparams-expr-function} - -Allows describing a lambda function for passing to a higher-order function. The left side of the arrow has a formal parameter, which is any ID, or multiple formal parameters – any IDs in a tuple. The right side of the arrow has an expression that can use these formal parameters, as well as any table columns. - -Exemple: `x -> 2 * x, str -> str != Referer.` - -Les fonctions d'ordre supérieur ne peuvent accepter que les fonctions lambda comme argument fonctionnel. - -Une fonction lambda qui accepte plusieurs arguments peuvent être passés à une fonction d'ordre supérieur. Dans ce cas, la fonction d'ordre supérieur est passé plusieurs tableaux de longueur identique que ces arguments correspondent. - -Pour certaines fonctions, telles que [arrayCount](#higher_order_functions-array-count) ou [arraySum](#higher_order_functions-array-count) le premier argument (la fonction lambda) peut être omis. Dans ce cas, un mappage identique est supposé. - -Une fonction lambda ne peut pas être omise pour les fonctions suivantes: - -- [arrayMap](#higher_order_functions-array-map) -- [arrayFilter](#higher_order_functions-array-filter) -- [arrayFill](#higher_order_functions-array-fill) -- [arrayReverseFill](#higher_order_functions-array-reverse-fill) -- [arraySplit](#higher_order_functions-array-split) -- [arrayReverseSplit](#higher_order_functions-array-reverse-split) -- [arrayFirst](#higher_order_functions-array-first) -- [arrayFirstIndex](#higher_order_functions-array-first-index) - -### arrayMap(func, arr1, …) {#higher_order_functions-array-map} - -Renvoie un tableau obtenu à partir de l'application d'origine `func` fonction à chaque élément dans le `arr` tableau. - -Exemple: - -``` sql -SELECT arrayMap(x -> (x + 2), [1, 2, 3]) as res; -``` - -``` text -┌─res─────┐ -│ [3,4,5] │ -└─────────┘ -``` - -L'exemple suivant montre comment créer un n-uplet d'éléments de différents tableaux: - -``` sql -SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res -``` - -``` text -┌─res─────────────────┐ -│ [(1,4),(2,5),(3,6)] │ -└─────────────────────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayMap` fonction. - -### arrayFilter(func, arr1, …) {#higher_order_functions-array-filter} - -Renvoie un tableau contenant uniquement les éléments `arr1` pour ce qui `func` retourne autre chose que 0. - -Exemple: - -``` sql -SELECT arrayFilter(x -> x LIKE '%World%', ['Hello', 'abc World']) AS res -``` - -``` text -┌─res───────────┐ -│ ['abc World'] │ -└───────────────┘ -``` - -``` sql -SELECT - arrayFilter( - (i, x) -> x LIKE '%World%', - arrayEnumerate(arr), - ['Hello', 'abc World'] AS arr) - AS res -``` - -``` text -┌─res─┐ -│ [2] │ -└─────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayFilter` fonction. - -### arrayFill(func, arr1, …) {#higher_order_functions-array-fill} - -Analyse par le biais de `arr1` du premier élément au dernier élément et remplacer `arr1[i]` par `arr1[i - 1]` si `func` renvoie 0. Le premier élément de `arr1` ne sera pas remplacé. - -Exemple: - -``` sql -SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res -``` - -``` text -┌─res──────────────────────────────┐ -│ [1,1,3,11,12,12,12,5,6,14,14,14] │ -└──────────────────────────────────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayFill` fonction. - -### arrayReverseFill(func, arr1, …) {#higher_order_functions-array-reverse-fill} - -Analyse par le biais de `arr1` du dernier élément au premier élément et remplacer `arr1[i]` par `arr1[i + 1]` si `func` renvoie 0. Le dernier élément de `arr1` ne sera pas remplacé. - -Exemple: - -``` sql -SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res -``` - -``` text -┌─res────────────────────────────────┐ -│ [1,3,3,11,12,5,5,5,6,14,NULL,NULL] │ -└────────────────────────────────────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayReverseFill` fonction. - -### arraySplit(func, arr1, …) {#higher_order_functions-array-split} - -Split `arr1` en plusieurs tableaux. Lorsque `func` retourne autre chose que 0, la matrice sera de split sur le côté gauche de l'élément. Le tableau ne sera pas partagé avant le premier élément. - -Exemple: - -``` sql -SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res -``` - -``` text -┌─res─────────────┐ -│ [[1,2,3],[4,5]] │ -└─────────────────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arraySplit` fonction. - -### arrayReverseSplit(func, arr1, …) {#higher_order_functions-array-reverse-split} - -Split `arr1` en plusieurs tableaux. Lorsque `func` retourne autre chose que 0, la matrice sera de split sur le côté droit de l'élément. Le tableau ne sera pas divisé après le dernier élément. - -Exemple: - -``` sql -SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res -``` - -``` text -┌─res───────────────┐ -│ [[1],[2,3,4],[5]] │ -└───────────────────┘ -``` - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arraySplit` fonction. - -### arrayCount(\[func,\] arr1, …) {#higher_order_functions-array-count} - -Renvoie le nombre d'éléments dans l'arr tableau pour lequel func renvoie autre chose que 0. Si ‘func’ n'est pas spécifié, il renvoie le nombre d'éléments non nuls dans le tableau. - -### arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1} - -Renvoie 1 s'il existe au moins un élément ‘arr’ pour ce qui ‘func’ retourne autre chose que 0. Sinon, il renvoie 0. - -### arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1} - -Renvoie 1 si ‘func’ retourne autre chose que 0 pour tous les éléments de ‘arr’. Sinon, il renvoie 0. - -### arraySum(\[func,\] arr1, …) {#higher-order-functions-array-sum} - -Renvoie la somme de la ‘func’ valeur. Si la fonction est omise, elle retourne la somme des éléments du tableau. - -### arrayFirst(func, arr1, …) {#higher_order_functions-array-first} - -Renvoie le premier élément du ‘arr1’ tableau pour lequel ‘func’ retourne autre chose que 0. - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayFirst` fonction. - -### arrayFirstIndex(func, arr1, …) {#higher_order_functions-array-first-index} - -Renvoie l'index du premier élément de la ‘arr1’ tableau pour lequel ‘func’ retourne autre chose que 0. - -Notez que le premier argument (fonction lambda) ne peut pas être omis dans le `arrayFirstIndex` fonction. - -### arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1} - -Retourne un tableau des sommes partielles d'éléments dans le tableau source (une somme). Si l' `func` la fonction est spécifiée, les valeurs des éléments du tableau sont convertis par cette fonction avant l'addition. - -Exemple: - -``` sql -SELECT arrayCumSum([1, 1, 1, 1]) AS res -``` - -``` text -┌─res──────────┐ -│ [1, 2, 3, 4] │ -└──────────────┘ -``` - -### arrayCumSumNonNegative (arr) {#arraycumsumnonnegativearr} - -Même que `arrayCumSum`, renvoie un tableau des sommes partielles d'éléments dans le tableau source (une somme). Différent `arrayCumSum`, lorsque la valeur renvoyée contient une valeur inférieure à zéro, la valeur est remplacée par zéro et le calcul ultérieur est effectué avec des paramètres zéro. Exemple: - -``` sql -SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res -``` - -``` text -┌─res───────┐ -│ [1,2,0,1] │ -└───────────┘ -``` - -### arraySort(\[func,\] arr1, …) {#arraysortfunc-arr1} - -Renvoie un tableau à la suite du tri des éléments de `arr1` dans l'ordre croissant. Si l' `func` la fonction est spécifiée, l'ordre de classement est déterminé par le résultat de la fonction `func` appliquée aux éléments du tableau (tableaux) - -Le [Transformation schwartzienne](https://en.wikipedia.org/wiki/Schwartzian_transform) est utilisé pour améliorer l'efficacité du tri. - -Exemple: - -``` sql -SELECT arraySort((x, y) -> y, ['hello', 'world'], [2, 1]); -``` - -``` text -┌─res────────────────┐ -│ ['world', 'hello'] │ -└────────────────────┘ -``` - -Pour plus d'informations sur la `arraySort` la méthode, voir l' [Fonctions pour travailler avec des tableaux](array-functions.md#array_functions-sort) section. - -### arrayReverseSort(\[func,\] arr1, …) {#arrayreversesortfunc-arr1} - -Renvoie un tableau à la suite du tri des éléments de `arr1` dans l'ordre décroissant. Si l' `func` la fonction est spécifiée, l'ordre de classement est déterminé par le résultat de la fonction `func` appliquée aux éléments du tableau (tableaux). - -Exemple: - -``` sql -SELECT arrayReverseSort((x, y) -> y, ['hello', 'world'], [2, 1]) as res; -``` - -``` text -┌─res───────────────┐ -│ ['hello','world'] │ -└───────────────────┘ -``` - -Pour plus d'informations sur la `arrayReverseSort` la méthode, voir l' [Fonctions pour travailler avec des tableaux](array-functions.md#array_functions-reverse-sort) section. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/higher_order_functions/) diff --git a/docs/fr/sql-reference/functions/in-functions.md b/docs/fr/sql-reference/functions/in-functions.md deleted file mode 100644 index ced5ef73e46..00000000000 --- a/docs/fr/sql-reference/functions/in-functions.md +++ /dev/null @@ -1,26 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 60 -toc_title: "Mise en \u0153uvre de L'op\xE9rateur IN" ---- - -# Fonctions de mise en œuvre de L'opérateur IN {#functions-for-implementing-the-in-operator} - -## in, notin, globalIn, globalNotIn {#in-functions} - -Voir la section [Dans les opérateurs](../operators/in.md#select-in-operators). - -## tuple(x, y, …), operator (x, y, …) {#tuplex-y-operator-x-y} - -Une fonction qui permet de regrouper plusieurs colonnes. -For columns with the types T1, T2, …, it returns a Tuple(T1, T2, …) type tuple containing these columns. There is no cost to execute the function. -Les Tuples sont normalement utilisés comme valeurs intermédiaires pour un argument D'opérateurs IN, ou pour créer une liste de paramètres formels de fonctions lambda. Les Tuples ne peuvent pas être écrits sur une table. - -## tupleElement (tuple, n), opérateur X. N {#tupleelementtuple-n-operator-x-n} - -Une fonction qui permet d'obtenir une colonne à partir d'un tuple. -‘N’ est l'index de colonne, à partir de 1. N doit être une constante. ‘N’ doit être une constante. ‘N’ doit être un entier postif strict ne dépassant pas la taille du tuple. -Il n'y a aucun coût pour exécuter la fonction. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/in_functions/) diff --git a/docs/fr/sql-reference/functions/index.md b/docs/fr/sql-reference/functions/index.md deleted file mode 100644 index 6e5333f68f5..00000000000 --- a/docs/fr/sql-reference/functions/index.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Fonction -toc_priority: 32 -toc_title: Introduction ---- - -# Fonction {#functions} - -Il y a au moins\* deux types de fonctions - des fonctions régulières (elles sont simplement appelées “functions”) and aggregate functions. These are completely different concepts. Regular functions work as if they are applied to each row separately (for each row, the result of the function doesn't depend on the other rows). Aggregate functions accumulate a set of values from various rows (i.e. they depend on the entire set of rows). - -Dans cette section, nous discutons des fonctions classiques. Pour les fonctions d'agrégation, voir la section “Aggregate functions”. - -\* - Il existe un troisième type de fonction ‘arrayJoin’ la fonction appartient à; les fonctions de table peuvent également être mentionnées séparément.\* - -## Typage Fort {#strong-typing} - -Contrairement à SQL standard, ClickHouse a une forte typage. En d'autres termes, il ne fait pas de conversions implicites entre les types. Chaque fonction fonctionne pour un ensemble spécifique de types. Cela signifie que vous devez parfois utiliser des fonctions de conversion de type. - -## Élimination Des Sous-Expressions Courantes {#common-subexpression-elimination} - -Toutes les expressions d'une requête qui ont le même AST (le même enregistrement ou le même résultat d'analyse syntaxique) sont considérées comme ayant des valeurs identiques. De telles expressions sont concaténées et exécutées une fois. Les sous-requêtes identiques sont également éliminées de cette façon. - -## Types de résultats {#types-of-results} - -Toutes les fonctions renvoient un seul retour comme résultat (pas plusieurs valeurs, et pas des valeurs nulles). Le type de résultat est généralement défini uniquement par les types d'arguments, pas par les valeurs. Les Exceptions sont la fonction tupleElement (l'opérateur A. N) et la fonction toFixedString. - -## Constant {#constants} - -Pour simplifier, certaines fonctions ne peuvent fonctionner qu'avec des constantes pour certains arguments. Par exemple, le bon argument de L'opérateur LIKE doit être une constante. -Presque toutes les fonctions renvoient une constante pour des arguments constants. L'exception est les fonctions qui génèrent des nombres aléatoires. -Le ‘now’ function renvoie des valeurs différentes pour les requêtes qui ont été exécutées à des moments différents, mais le résultat est considéré comme une constante, car la constance n'est importante que dans une seule requête. -Une expression constante est également considérée comme une constante (par exemple, la moitié droite de L'opérateur LIKE peut être construite à partir de plusieurs constantes). - -Les fonctions peuvent être implémentées de différentes manières pour des arguments constants et non constants (un code différent est exécuté). Mais les résultats pour une constante et pour une colonne vraie Ne contenant que la même valeur doivent correspondre les uns aux autres. - -## Le Traitement NULL {#null-processing} - -Les fonctions ont les comportements suivants: - -- Si au moins l'un des arguments de la fonction est `NULL` le résultat de la fonction est également `NULL`. -- Comportement spécial spécifié individuellement dans la description de chaque fonction. Dans le code source de ClickHouse, ces fonctions ont `UseDefaultImplementationForNulls=false`. - -## Constance {#constancy} - -Functions can't change the values of their arguments – any changes are returned as the result. Thus, the result of calculating separate functions does not depend on the order in which the functions are written in the query. - -## Erreur De Manipulation {#error-handling} - -Certaines fonctions peuvent lancer une exception si les données ne sont pas valides. Dans ce cas, la requête est annulée et un message d'erreur est retourné au client. Pour le traitement distribué, lorsqu'une exception se produit sur l'un des serveurs, les autres serveurs aussi tenté d'interrompre la requête. - -## Évaluation des Expressions D'Argument {#evaluation-of-argument-expressions} - -Dans presque tous les langages de programmation, l'un des arguments peut pas être évalué pour certains opérateurs. Ce sont généralement les opérateurs `&&`, `||`, et `?:`. -Mais dans ClickHouse, les arguments des fonctions (opérateurs) sont toujours évalués. En effet, des parties entières de colonnes sont évaluées à la fois, au lieu de calculer chaque ligne séparément. - -## Exécution de fonctions pour le traitement de requêtes distribuées {#performing-functions-for-distributed-query-processing} - -Pour le traitement de requête distribué, autant d'étapes de traitement de requête que possible sont effectuées sur des serveurs distants, et le reste des étapes (fusion des résultats intermédiaires et tout ce qui suit) sont effectuées sur le serveur demandeur. - -Cela signifie que les fonctions peuvent être effectuées sur différents serveurs. -Par exemple, dans la requête `SELECT f(sum(g(x))) FROM distributed_table GROUP BY h(y),` - -- si un `distributed_table` a au moins deux fragments, les fonctions ‘g’ et ‘h’ sont effectuées sur des serveurs distants, et la fonction ‘f’ est effectuée sur le serveur demandeur. -- si un `distributed_table` a un seul fragment, tous les ‘f’, ‘g’, et ‘h’ les fonctions sont exécutées sur le serveur de ce fragment. - -Le résultat d'une fonction habituellement ne dépendent pas le serveur sur lequel elle est exécutée. Cependant, parfois c'est important. -Par exemple, les fonctions qui fonctionnent avec des dictionnaires utilisent le dictionnaire qui existe sur le serveur sur lequel elles s'exécutent. -Un autre exemple est l' `hostName` fonction, qui renvoie le nom du serveur sur lequel il s'exécute afin de `GROUP BY` par les serveurs dans un `SELECT` requête. - -Si une fonction dans une requête est effectuée sur le demandeur serveur, mais vous devez l'exécuter sur des serveurs distants, vous pouvez l'envelopper dans un ‘any’ fonction d'agrégation ou l'ajouter à une clé dans `GROUP BY`. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/) diff --git a/docs/fr/sql-reference/functions/introspection.md b/docs/fr/sql-reference/functions/introspection.md deleted file mode 100644 index 91299217dc7..00000000000 --- a/docs/fr/sql-reference/functions/introspection.md +++ /dev/null @@ -1,310 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 65 -toc_title: Introspection ---- - -# Fonctions D'Introspection {#introspection-functions} - -Vous pouvez utiliser les fonctions décrites dans ce chapitre pour introspecter [ELF](https://en.wikipedia.org/wiki/Executable_and_Linkable_Format) et [DWARF](https://en.wikipedia.org/wiki/DWARF) pour le profilage de requête. - -!!! warning "Avertissement" - Ces fonctions sont lentes et peuvent imposer des considérations de sécurité. - -Pour le bon fonctionnement des fonctions d'introspection: - -- Installer le `clickhouse-common-static-dbg` paquet. - -- Définir le [allow_introspection_functions](../../operations/settings/settings.md#settings-allow_introspection_functions) réglage sur 1. - - For security reasons introspection functions are disabled by default. - -Clickhouse enregistre les rapports du profileur [trace_log](../../operations/system-tables.md#system_tables-trace_log) système de table. Assurez-vous que la table et le profileur sont correctement configurés. - -## addressToLine {#addresstoline} - -Convertit l'adresse de mémoire virtuelle dans le processus de serveur ClickHouse en nom de fichier et en numéro de ligne dans le code source de ClickHouse. - -Si vous utilisez des paquets clickhouse officiels, vous devez installer le `clickhouse-common-static-dbg` paquet. - -**Syntaxe** - -``` sql -addressToLine(address_of_binary_instruction) -``` - -**Paramètre** - -- `address_of_binary_instruction` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Address of instruction in a running process. - -**Valeur renvoyée** - -- Nom de fichier du code Source et le numéro de ligne dans ce fichier délimité par deux-points. - - For example, `/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:199`, where `199` is a line number. - -- Nom d'un binaire, si la fonction n'a pas pu trouver les informations de débogage. - -- Chaîne vide, si l'adresse n'est pas valide. - -Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -Activation des fonctions d'introspection: - -``` sql -SET allow_introspection_functions=1 -``` - -Sélection de la première chaîne de `trace_log` système de table: - -``` sql -SELECT * FROM system.trace_log LIMIT 1 \G -``` - -``` text -Row 1: -────── -event_date: 2019-11-19 -event_time: 2019-11-19 18:57:23 -revision: 54429 -timer_type: Real -thread_number: 48 -query_id: 421b6855-1858-45a5-8f37-f383409d6d72 -trace: [140658411141617,94784174532828,94784076370703,94784076372094,94784076361020,94784175007680,140658411116251,140658403895439] -``` - -Le `trace` champ contient la trace de pile au moment de l'échantillonnage. - -Obtenir le nom de fichier du code source et le numéro de ligne pour une seule adresse: - -``` sql -SELECT addressToLine(94784076370703) \G -``` - -``` text -Row 1: -────── -addressToLine(94784076370703): /build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:199 -``` - -Application de la fonction à la trace de la pile entière: - -``` sql -SELECT - arrayStringConcat(arrayMap(x -> addressToLine(x), trace), '\n') AS trace_source_code_lines -FROM system.trace_log -LIMIT 1 -\G -``` - -Le [arrayMap](higher-order-functions.md#higher_order_functions-array-map) permet de traiter chaque élément individuel de l' `trace` tableau par la `addressToLine` fonction. Le résultat de ce traitement que vous voyez dans l' `trace_source_code_lines` colonne de sortie. - -``` text -Row 1: -────── -trace_source_code_lines: /lib/x86_64-linux-gnu/libpthread-2.27.so -/usr/lib/debug/usr/bin/clickhouse -/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:199 -/build/obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:155 -/usr/include/c++/9/bits/atomic_base.h:551 -/usr/lib/debug/usr/bin/clickhouse -/lib/x86_64-linux-gnu/libpthread-2.27.so -/build/glibc-OTsEL5/glibc-2.27/misc/../sysdeps/unix/sysv/linux/x86_64/clone.S:97 -``` - -## adressetosymbol {#addresstosymbol} - -Convertit l'adresse de mémoire virtuelle dans le processus de serveur ClickHouse en symbole à partir des fichiers d'objets ClickHouse. - -**Syntaxe** - -``` sql -addressToSymbol(address_of_binary_instruction) -``` - -**Paramètre** - -- `address_of_binary_instruction` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Address of instruction in a running process. - -**Valeur renvoyée** - -- Symbole des fichiers D'objets ClickHouse. -- Chaîne vide, si l'adresse n'est pas valide. - -Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -Activation des fonctions d'introspection: - -``` sql -SET allow_introspection_functions=1 -``` - -Sélection de la première chaîne de `trace_log` système de table: - -``` sql -SELECT * FROM system.trace_log LIMIT 1 \G -``` - -``` text -Row 1: -────── -event_date: 2019-11-20 -event_time: 2019-11-20 16:57:59 -revision: 54429 -timer_type: Real -thread_number: 48 -query_id: 724028bf-f550-45aa-910d-2af6212b94ac -trace: [94138803686098,94138815010911,94138815096522,94138815101224,94138815102091,94138814222988,94138806823642,94138814457211,94138806823642,94138814457211,94138806823642,94138806795179,94138806796144,94138753770094,94138753771646,94138753760572,94138852407232,140399185266395,140399178045583] -``` - -Le `trace` champ contient la trace de pile au moment de l'échantillonnage. - -Obtenir un symbole pour une seule adresse: - -``` sql -SELECT addressToSymbol(94138803686098) \G -``` - -``` text -Row 1: -────── -addressToSymbol(94138803686098): _ZNK2DB24IAggregateFunctionHelperINS_20AggregateFunctionSumImmNS_24AggregateFunctionSumDataImEEEEE19addBatchSinglePlaceEmPcPPKNS_7IColumnEPNS_5ArenaE -``` - -Application de la fonction à la trace de la pile entière: - -``` sql -SELECT - arrayStringConcat(arrayMap(x -> addressToSymbol(x), trace), '\n') AS trace_symbols -FROM system.trace_log -LIMIT 1 -\G -``` - -Le [arrayMap](higher-order-functions.md#higher_order_functions-array-map) permet de traiter chaque élément individuel de l' `trace` tableau par la `addressToSymbols` fonction. Le résultat de ce traitement que vous voyez dans l' `trace_symbols` colonne de sortie. - -``` text -Row 1: -────── -trace_symbols: _ZNK2DB24IAggregateFunctionHelperINS_20AggregateFunctionSumImmNS_24AggregateFunctionSumDataImEEEEE19addBatchSinglePlaceEmPcPPKNS_7IColumnEPNS_5ArenaE -_ZNK2DB10Aggregator21executeWithoutKeyImplERPcmPNS0_28AggregateFunctionInstructionEPNS_5ArenaE -_ZN2DB10Aggregator14executeOnBlockESt6vectorIN3COWINS_7IColumnEE13immutable_ptrIS3_EESaIS6_EEmRNS_22AggregatedDataVariantsERS1_IPKS3_SaISC_EERS1_ISE_SaISE_EERb -_ZN2DB10Aggregator14executeOnBlockERKNS_5BlockERNS_22AggregatedDataVariantsERSt6vectorIPKNS_7IColumnESaIS9_EERS6_ISB_SaISB_EERb -_ZN2DB10Aggregator7executeERKSt10shared_ptrINS_17IBlockInputStreamEERNS_22AggregatedDataVariantsE -_ZN2DB27AggregatingBlockInputStream8readImplEv -_ZN2DB17IBlockInputStream4readEv -_ZN2DB26ExpressionBlockInputStream8readImplEv -_ZN2DB17IBlockInputStream4readEv -_ZN2DB26ExpressionBlockInputStream8readImplEv -_ZN2DB17IBlockInputStream4readEv -_ZN2DB28AsynchronousBlockInputStream9calculateEv -_ZNSt17_Function_handlerIFvvEZN2DB28AsynchronousBlockInputStream4nextEvEUlvE_E9_M_invokeERKSt9_Any_data -_ZN14ThreadPoolImplI20ThreadFromGlobalPoolE6workerESt14_List_iteratorIS0_E -_ZZN20ThreadFromGlobalPoolC4IZN14ThreadPoolImplIS_E12scheduleImplIvEET_St8functionIFvvEEiSt8optionalImEEUlvE1_JEEEOS4_DpOT0_ENKUlvE_clEv -_ZN14ThreadPoolImplISt6threadE6workerESt14_List_iteratorIS0_E -execute_native_thread_routine -start_thread -clone -``` - -## demangle {#demangle} - -Convertit un symbole que vous pouvez obtenir en utilisant le [adressetosymbol](#addresstosymbol) fonction au nom de la fonction c++. - -**Syntaxe** - -``` sql -demangle(symbol) -``` - -**Paramètre** - -- `symbol` ([Chaîne](../../sql-reference/data-types/string.md)) — Symbol from an object file. - -**Valeur renvoyée** - -- Nom de la fonction C++. -- Chaîne vide si un symbole n'est pas valide. - -Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -Activation des fonctions d'introspection: - -``` sql -SET allow_introspection_functions=1 -``` - -Sélection de la première chaîne de `trace_log` système de table: - -``` sql -SELECT * FROM system.trace_log LIMIT 1 \G -``` - -``` text -Row 1: -────── -event_date: 2019-11-20 -event_time: 2019-11-20 16:57:59 -revision: 54429 -timer_type: Real -thread_number: 48 -query_id: 724028bf-f550-45aa-910d-2af6212b94ac -trace: [94138803686098,94138815010911,94138815096522,94138815101224,94138815102091,94138814222988,94138806823642,94138814457211,94138806823642,94138814457211,94138806823642,94138806795179,94138806796144,94138753770094,94138753771646,94138753760572,94138852407232,140399185266395,140399178045583] -``` - -Le `trace` champ contient la trace de pile au moment de l'échantillonnage. - -Obtenir un nom de fonction pour une seule adresse: - -``` sql -SELECT demangle(addressToSymbol(94138803686098)) \G -``` - -``` text -Row 1: -────── -demangle(addressToSymbol(94138803686098)): DB::IAggregateFunctionHelper > >::addBatchSinglePlace(unsigned long, char*, DB::IColumn const**, DB::Arena*) const -``` - -Application de la fonction à la trace de la pile entière: - -``` sql -SELECT - arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') AS trace_functions -FROM system.trace_log -LIMIT 1 -\G -``` - -Le [arrayMap](higher-order-functions.md#higher_order_functions-array-map) permet de traiter chaque élément individuel de l' `trace` tableau par la `demangle` fonction. Le résultat de ce traitement que vous voyez dans l' `trace_functions` colonne de sortie. - -``` text -Row 1: -────── -trace_functions: DB::IAggregateFunctionHelper > >::addBatchSinglePlace(unsigned long, char*, DB::IColumn const**, DB::Arena*) const -DB::Aggregator::executeWithoutKeyImpl(char*&, unsigned long, DB::Aggregator::AggregateFunctionInstruction*, DB::Arena*) const -DB::Aggregator::executeOnBlock(std::vector::immutable_ptr, std::allocator::immutable_ptr > >, unsigned long, DB::AggregatedDataVariants&, std::vector >&, std::vector >, std::allocator > > >&, bool&) -DB::Aggregator::executeOnBlock(DB::Block const&, DB::AggregatedDataVariants&, std::vector >&, std::vector >, std::allocator > > >&, bool&) -DB::Aggregator::execute(std::shared_ptr const&, DB::AggregatedDataVariants&) -DB::AggregatingBlockInputStream::readImpl() -DB::IBlockInputStream::read() -DB::ExpressionBlockInputStream::readImpl() -DB::IBlockInputStream::read() -DB::ExpressionBlockInputStream::readImpl() -DB::IBlockInputStream::read() -DB::AsynchronousBlockInputStream::calculate() -std::_Function_handler::_M_invoke(std::_Any_data const&) -ThreadPoolImpl::worker(std::_List_iterator) -ThreadFromGlobalPool::ThreadFromGlobalPool::scheduleImpl(std::function, int, std::optional)::{lambda()#3}>(ThreadPoolImpl::scheduleImpl(std::function, int, std::optional)::{lambda()#3}&&)::{lambda()#1}::operator()() const -ThreadPoolImpl::worker(std::_List_iterator) -execute_native_thread_routine -start_thread -clone -``` diff --git a/docs/fr/sql-reference/functions/ip-address-functions.md b/docs/fr/sql-reference/functions/ip-address-functions.md deleted file mode 100644 index 8beb40a534b..00000000000 --- a/docs/fr/sql-reference/functions/ip-address-functions.md +++ /dev/null @@ -1,248 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 55 -toc_title: Travailler avec des adresses IP ---- - -# Fonctions pour travailler avec des adresses IP {#functions-for-working-with-ip-addresses} - -## IPv4NumToString (num) {#ipv4numtostringnum} - -Prend un numéro UInt32. Interprète comme une adresse IPv4 dans big endian. Renvoie une chaîne contenant l'adresse IPv4 correspondante au format A. B. C. d (Nombres séparés par des points sous forme décimale). - -## IPv4StringToNum (s) {#ipv4stringtonums} - -La fonction inverse de IPv4NumToString. Si L'adresse IPv4 a un format non valide, elle renvoie 0. - -## IPv4NumToStringClassC(num) {#ipv4numtostringclasscnum} - -Similaire à IPv4NumToString, mais en utilisant xxx au lieu du dernier octet. - -Exemple: - -``` sql -SELECT - IPv4NumToStringClassC(ClientIP) AS k, - count() AS c -FROM test.hits -GROUP BY k -ORDER BY c DESC -LIMIT 10 -``` - -``` text -┌─k──────────────┬─────c─┐ -│ 83.149.9.xxx │ 26238 │ -│ 217.118.81.xxx │ 26074 │ -│ 213.87.129.xxx │ 25481 │ -│ 83.149.8.xxx │ 24984 │ -│ 217.118.83.xxx │ 22797 │ -│ 78.25.120.xxx │ 22354 │ -│ 213.87.131.xxx │ 21285 │ -│ 78.25.121.xxx │ 20887 │ -│ 188.162.65.xxx │ 19694 │ -│ 83.149.48.xxx │ 17406 │ -└────────────────┴───────┘ -``` - -Depuis l'utilisation de ‘xxx’ est très inhabituel, cela peut être changé à l'avenir. Nous vous recommandons de ne pas compter sur le format exact de ce fragment. - -### IPv6NumToString (x) {#ipv6numtostringx} - -Accepte une valeur FixedString (16) contenant L'adresse IPv6 au format binaire. Renvoie une chaîne contenant cette adresse au format texte. -Les adresses IPv4 mappées IPv6 sont sorties au format:: ffff: 111.222.33.44. Exemple: - -``` sql -SELECT IPv6NumToString(toFixedString(unhex('2A0206B8000000000000000000000011'), 16)) AS addr -``` - -``` text -┌─addr─────────┐ -│ 2a02:6b8::11 │ -└──────────────┘ -``` - -``` sql -SELECT - IPv6NumToString(ClientIP6 AS k), - count() AS c -FROM hits_all -WHERE EventDate = today() AND substring(ClientIP6, 1, 12) != unhex('00000000000000000000FFFF') -GROUP BY k -ORDER BY c DESC -LIMIT 10 -``` - -``` text -┌─IPv6NumToString(ClientIP6)──────────────┬─────c─┐ -│ 2a02:2168:aaa:bbbb::2 │ 24695 │ -│ 2a02:2698:abcd:abcd:abcd:abcd:8888:5555 │ 22408 │ -│ 2a02:6b8:0:fff::ff │ 16389 │ -│ 2a01:4f8:111:6666::2 │ 16016 │ -│ 2a02:2168:888:222::1 │ 15896 │ -│ 2a01:7e00::ffff:ffff:ffff:222 │ 14774 │ -│ 2a02:8109:eee:ee:eeee:eeee:eeee:eeee │ 14443 │ -│ 2a02:810b:8888:888:8888:8888:8888:8888 │ 14345 │ -│ 2a02:6b8:0:444:4444:4444:4444:4444 │ 14279 │ -│ 2a01:7e00::ffff:ffff:ffff:ffff │ 13880 │ -└─────────────────────────────────────────┴───────┘ -``` - -``` sql -SELECT - IPv6NumToString(ClientIP6 AS k), - count() AS c -FROM hits_all -WHERE EventDate = today() -GROUP BY k -ORDER BY c DESC -LIMIT 10 -``` - -``` text -┌─IPv6NumToString(ClientIP6)─┬──────c─┐ -│ ::ffff:94.26.111.111 │ 747440 │ -│ ::ffff:37.143.222.4 │ 529483 │ -│ ::ffff:5.166.111.99 │ 317707 │ -│ ::ffff:46.38.11.77 │ 263086 │ -│ ::ffff:79.105.111.111 │ 186611 │ -│ ::ffff:93.92.111.88 │ 176773 │ -│ ::ffff:84.53.111.33 │ 158709 │ -│ ::ffff:217.118.11.22 │ 154004 │ -│ ::ffff:217.118.11.33 │ 148449 │ -│ ::ffff:217.118.11.44 │ 148243 │ -└────────────────────────────┴────────┘ -``` - -## IPv6StringToNum (s) {#ipv6stringtonums} - -La fonction inverse de IPv6NumToString. Si L'adresse IPv6 a un format non valide, elle renvoie une chaîne d'octets null. -HEX peut être en majuscules ou en minuscules. - -## IPv4ToIPv6 (x) {#ipv4toipv6x} - -Prend un `UInt32` nombre. Interprète comme une adresse IPv4 dans [big endian](https://en.wikipedia.org/wiki/Endianness). Retourne un `FixedString(16)` valeur contenant l'adresse IPv6 au format binaire. Exemple: - -``` sql -SELECT IPv6NumToString(IPv4ToIPv6(IPv4StringToNum('192.168.0.1'))) AS addr -``` - -``` text -┌─addr───────────────┐ -│ ::ffff:192.168.0.1 │ -└────────────────────┘ -``` - -## cutIPv6 (x, bytesToCutForIPv6, bytesToCutForIPv4) {#cutipv6x-bytestocutforipv6-bytestocutforipv4} - -Accepte une valeur FixedString (16) contenant L'adresse IPv6 au format binaire. Renvoie une chaîne contenant l'adresse du nombre spécifié d'octets retiré au format texte. Exemple: - -``` sql -WITH - IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D') AS ipv6, - IPv4ToIPv6(IPv4StringToNum('192.168.0.1')) AS ipv4 -SELECT - cutIPv6(ipv6, 2, 0), - cutIPv6(ipv4, 0, 2) -``` - -``` text -┌─cutIPv6(ipv6, 2, 0)─────────────────┬─cutIPv6(ipv4, 0, 2)─┐ -│ 2001:db8:ac10:fe01:feed:babe:cafe:0 │ ::ffff:192.168.0.0 │ -└─────────────────────────────────────┴─────────────────────┘ -``` - -## Ipv4cirtorange (ipv4, Cidr), {#ipv4cidrtorangeipv4-cidr} - -Accepte un IPv4 et une valeur UInt8 contenant [CIDR](https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing). Renvoie un tuple avec deux IPv4 contenant la plage inférieure et la plage supérieure du sous-réseau. - -``` sql -SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 16) -``` - -``` text -┌─IPv4CIDRToRange(toIPv4('192.168.5.2'), 16)─┐ -│ ('192.168.0.0','192.168.255.255') │ -└────────────────────────────────────────────┘ -``` - -## Ipv6cirtorange (ipv6, Cidr), {#ipv6cidrtorangeipv6-cidr} - -Accepte un IPv6 et une valeur UInt8 contenant le CIDR. Renvoie un tuple avec deux IPv6 contenant la plage inférieure et la plage supérieure du sous-réseau. - -``` sql -SELECT IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32); -``` - -``` text -┌─IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐ -│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff') │ -└────────────────────────────────────────────────────────────────────────┘ -``` - -## toipv4 (chaîne) {#toipv4string} - -Un alias `IPv4StringToNum()` cela prend une forme de chaîne D'adresse IPv4 et renvoie la valeur de [IPv4](../../sql-reference/data-types/domains/ipv4.md) type, qui est binaire égal à la valeur renvoyée par `IPv4StringToNum()`. - -``` sql -WITH - '171.225.130.45' as IPv4_string -SELECT - toTypeName(IPv4StringToNum(IPv4_string)), - toTypeName(toIPv4(IPv4_string)) -``` - -``` text -┌─toTypeName(IPv4StringToNum(IPv4_string))─┬─toTypeName(toIPv4(IPv4_string))─┐ -│ UInt32 │ IPv4 │ -└──────────────────────────────────────────┴─────────────────────────────────┘ -``` - -``` sql -WITH - '171.225.130.45' as IPv4_string -SELECT - hex(IPv4StringToNum(IPv4_string)), - hex(toIPv4(IPv4_string)) -``` - -``` text -┌─hex(IPv4StringToNum(IPv4_string))─┬─hex(toIPv4(IPv4_string))─┐ -│ ABE1822D │ ABE1822D │ -└───────────────────────────────────┴──────────────────────────┘ -``` - -## toipv6 (chaîne) {#toipv6string} - -Un alias `IPv6StringToNum()` cela prend une forme de chaîne D'adresse IPv6 et renvoie la valeur de [IPv6](../../sql-reference/data-types/domains/ipv6.md) type, qui est binaire égal à la valeur renvoyée par `IPv6StringToNum()`. - -``` sql -WITH - '2001:438:ffff::407d:1bc1' as IPv6_string -SELECT - toTypeName(IPv6StringToNum(IPv6_string)), - toTypeName(toIPv6(IPv6_string)) -``` - -``` text -┌─toTypeName(IPv6StringToNum(IPv6_string))─┬─toTypeName(toIPv6(IPv6_string))─┐ -│ FixedString(16) │ IPv6 │ -└──────────────────────────────────────────┴─────────────────────────────────┘ -``` - -``` sql -WITH - '2001:438:ffff::407d:1bc1' as IPv6_string -SELECT - hex(IPv6StringToNum(IPv6_string)), - hex(toIPv6(IPv6_string)) -``` - -``` text -┌─hex(IPv6StringToNum(IPv6_string))─┬─hex(toIPv6(IPv6_string))─────────┐ -│ 20010438FFFF000000000000407D1BC1 │ 20010438FFFF000000000000407D1BC1 │ -└───────────────────────────────────┴──────────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/ip_address_functions/) diff --git a/docs/fr/sql-reference/functions/json-functions.md b/docs/fr/sql-reference/functions/json-functions.md deleted file mode 100644 index 5f92c99d0f5..00000000000 --- a/docs/fr/sql-reference/functions/json-functions.md +++ /dev/null @@ -1,297 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 56 -toc_title: Travailler avec JSON ---- - -# Fonctions pour travailler avec JSON {#functions-for-working-with-json} - -Dans Yandex.Metrica, JSON est transmis par les utilisateurs en tant que paramètres de session. Il y a quelques fonctions spéciales pour travailler avec ce JSON. (Bien que dans la plupart des cas, les JSONs soient en outre prétraités et les valeurs résultantes sont placées dans des colonnes séparées dans leur format traité.) Toutes ces fonctions sont basées sur des hypothèses fortes sur ce que le JSON peut être, mais elles essaient de faire le moins possible pour faire le travail. - -Les hypothèses suivantes sont apportées: - -1. Le nom du champ (argument de fonction) doit être une constante. -2. Le nom du champ est en quelque sorte codé canoniquement dans JSON. Exemple: `visitParamHas('{"abc":"def"}', 'abc') = 1`, mais `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` -3. Les champs sont recherchés à n'importe quel niveau d'imbrication, sans discrimination. S'il y a plusieurs champs correspondants, la première occurrence est utilisé. -4. Le JSON n'a pas de caractères d'espace en dehors des littéraux de chaîne. - -## visitParamHas(params, nom) {#visitparamhasparams-name} - -Vérifie s'il existe un champ avec ‘name’ nom. - -## visitParamExtractUInt(params, nom) {#visitparamextractuintparams-name} - -Analyse UInt64 à partir de la valeur du champ nommé ‘name’. Si c'est un champ de type chaîne, il tente d'analyser un numéro à partir du début de la chaîne. Si le champ n'existe pas, ou s'il existe mais ne contient pas de nombre, il renvoie 0. - -## visitParamExtractInt(params, name) {#visitparamextractintparams-name} - -Le même que pour Int64. - -## visitParamExtractFloat(params, nom) {#visitparamextractfloatparams-name} - -Le même que pour Float64. - -## visitParamExtractBool(params, nom) {#visitparamextractboolparams-name} - -Analyse d'une valeur vrai/faux. Le résultat est UInt8. - -## visitParamExtractRaw(params, nom) {#visitparamextractrawparams-name} - -Retourne la valeur d'un champ, y compris les séparateurs. - -Exemple: - -``` sql -visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"' -visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' -``` - -## visitParamExtractString(params, nom) {#visitparamextractstringparams-name} - -Analyse la chaîne entre guillemets doubles. La valeur est sans échappement. Si l'échappement échoue, il renvoie une chaîne vide. - -Exemple: - -``` sql -visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' -visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' -visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' -visitParamExtractString('{"abc":"hello}', 'abc') = '' -``` - -Il n'y a actuellement aucun support pour les points de code dans le format `\uXXXX\uYYYY` qui ne proviennent pas du plan multilingue de base (ils sont convertis en CESU-8 au lieu de UTF-8). - -Les fonctions suivantes sont basées sur [simdjson](https://github.com/lemire/simdjson) conçu pour des exigences D'analyse JSON plus complexes. L'hypothèse 2 mentionnée ci-dessus s'applique toujours. - -## isValidJSON (json) {#isvalidjsonjson} - -Vérifie que la chaîne est un json valide. - -Exemple: - -``` sql -SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 -SELECT isValidJSON('not a json') = 0 -``` - -## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} - -Si la valeur existe dans le document JSON, `1` sera retourné. - -Si la valeur n'existe pas, `0` sera retourné. - -Exemple: - -``` sql -SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 1 -SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4) = 0 -``` - -`indices_or_keys` est une liste de zéro ou plusieurs arguments chacun d'entre eux peut être une chaîne ou un entier. - -- String = membre d'objet d'accès par clé. -- Entier positif = accédez au n-ème membre / clé depuis le début. -- Entier négatif = accédez au n-ème membre / clé à partir de la fin. - -Minimum de l'indice de l'élément est 1. Ainsi, l'élément 0 n'existe pas. - -Vous pouvez utiliser des entiers pour accéder à la fois aux tableaux JSON et aux objets JSON. - -Ainsi, par exemple: - -``` sql -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'a' -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', 2) = 'b' -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -1) = 'b' -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' -SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' -``` - -## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} - -Renvoie la longueur D'un tableau JSON ou d'un objet JSON. - -Si la valeur n'existe pas ou a un mauvais type, `0` sera retourné. - -Exemple: - -``` sql -SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 -SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 -``` - -## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} - -De retour le type d'une valeur JSON. - -Si la valeur n'existe pas, `Null` sera retourné. - -Exemple: - -``` sql -SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}') = 'Object' -SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' -SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' -``` - -## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} - -## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} - -## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} - -## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} - -Analyse un JSON et extrait une valeur. Ces fonctions sont similaires à `visitParam` fonction. - -Si la valeur n'existe pas ou a un mauvais type, `0` sera retourné. - -Exemple: - -``` sql -SELECT JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 -SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200.0 -SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 -``` - -## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} - -Analyse un JSON et extrait une chaîne. Cette fonction est similaire à `visitParamExtractString` fonction. - -Si la valeur n'existe pas ou a un mauvais type, une chaîne vide est retournée. - -La valeur est sans échappement. Si l'échappement échoue, il renvoie une chaîne vide. - -Exemple: - -``` sql -SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'hello' -SELECT JSONExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' -SELECT JSONExtractString('{"abc":"\\u263a"}', 'abc') = '☺' -SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' -SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' -``` - -## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} - -Analyse un JSON et extrait une valeur du type de données clickhouse donné. - -C'est une généralisation de la précédente `JSONExtract` fonction. -Cela signifie -`JSONExtract(..., 'String')` retourne exactement le même que `JSONExtractString()`, -`JSONExtract(..., 'Float64')` retourne exactement le même que `JSONExtractFloat()`. - -Exemple: - -``` sql -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(String, Array(Float64))') = ('hello',[-100,200,300]) -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(b Array(Float64), a String)') = ([-100,200,300],'hello') -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Nullable(Int8))') = [-100, NULL, NULL] -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4, 'Nullable(Int64)') = NULL -SELECT JSONExtract('{"passed": true}', 'passed', 'UInt8') = 1 -SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Thursday' -SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' -``` - -## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} - -Analyse les paires clé-valeur à partir D'un JSON où les valeurs sont du type de données clickhouse donné. - -Exemple: - -``` sql -SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)] -``` - -## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} - -Renvoie une partie de JSON en tant que chaîne non analysée. - -Si la pièce n'existe pas ou a un mauvais type, une chaîne vide est retournée. - -Exemple: - -``` sql -SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]' -``` - -## JSONExtractArrayRaw(json\[, indices_or_keys…\]) {#jsonextractarrayrawjson-indices-or-keys} - -Retourne un tableau avec des éléments de tableau JSON, chacun représenté comme une chaîne non analysée. - -Si la pièce n'existe pas ou n'est pas de tableau, un tableau vide sera retournée. - -Exemple: - -``` sql -SELECT JSONExtractArrayRaw('{"a": "hello", "b": [-100, 200.0, "hello"]}', 'b') = ['-100', '200.0', '"hello"']' -``` - -## JSONExtractKeysAndValuesRaw {#json-extract-keys-and-values-raw} - -Extrait les données brutes d'un objet JSON. - -**Syntaxe** - -``` sql -JSONExtractKeysAndValuesRaw(json[, p, a, t, h]) -``` - -**Paramètre** - -- `json` — [Chaîne](../data-types/string.md) avec JSON valide. -- `p, a, t, h` — Comma-separated indices or keys that specify the path to the inner field in a nested JSON object. Each argument can be either a [chaîne](../data-types/string.md) pour obtenir le champ par la touche ou un [entier](../data-types/int-uint.md) pour obtenir le N-ème champ (indexé à partir de 1, les entiers négatifs comptent à partir de la fin). S'il n'est pas défini, le JSON entier est analysé en tant qu'objet de niveau supérieur. Paramètre facultatif. - -**Valeurs renvoyées** - -- Tableau avec `('key', 'value')` tuple. Les deux membres du tuple sont des chaînes. -- Tableau vide si l'objet demandé n'existe pas, ou entrée JSON n'est pas valide. - -Type: [Tableau](../data-types/array.md)([Tuple](../data-types/tuple.md)([Chaîne](../data-types/string.md), [Chaîne](../data-types/string.md)). - -**Exemple** - -Requête: - -``` sql -SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}') -``` - -Résultat: - -``` text -┌─JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}')─┐ -│ [('a','[-100,200]'),('b','{"c":{"d":"hello","f":"world"}}')] │ -└──────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -Requête: - -``` sql -SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}', 'b') -``` - -Résultat: - -``` text -┌─JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}', 'b')─┐ -│ [('c','{"d":"hello","f":"world"}')] │ -└───────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -Requête: - -``` sql -SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}', -1, 'c') -``` - -Résultat: - -``` text -┌─JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello", "f": "world"}}}', -1, 'c')─┐ -│ [('d','"hello"'),('f','"world"')] │ -└───────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/json_functions/) diff --git a/docs/fr/sql-reference/functions/logical-functions.md b/docs/fr/sql-reference/functions/logical-functions.md deleted file mode 100644 index d01d9e02088..00000000000 --- a/docs/fr/sql-reference/functions/logical-functions.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: Logique ---- - -# Les Fonctions Logiques {#logical-functions} - -Les fonctions logiques acceptent tous les types numériques, mais renvoient un nombre UInt8 égal à 0 ou 1. - -Zéro comme argument est considéré “false,” alors que toute valeur non nulle est considérée comme “true”. - -## et, et opérateur {#and-and-operator} - -## ou, ou opérateur {#or-or-operator} - -## pas, pas opérateur {#not-not-operator} - -## xor {#xor} - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/logical_functions/) diff --git a/docs/fr/sql-reference/functions/machine-learning-functions.md b/docs/fr/sql-reference/functions/machine-learning-functions.md deleted file mode 100644 index 2212e0caa5a..00000000000 --- a/docs/fr/sql-reference/functions/machine-learning-functions.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 64 -toc_title: Fonctions D'Apprentissage Automatique ---- - -# Fonctions D'Apprentissage Automatique {#machine-learning-functions} - -## evalMLMethod (prédiction) {#machine_learning_methods-evalmlmethod} - -Prédiction utilisant des modèles de régression ajustés utilise `evalMLMethod` fonction. Voir le lien dans la `linearRegression`. - -### Régression Linéaire Stochastique {#stochastic-linear-regression} - -Le [stochasticLinearRegression](../../sql-reference/aggregate-functions/reference.md#agg_functions-stochasticlinearregression) la fonction d'agrégat implémente une méthode de descente de gradient stochastique utilisant un modèle linéaire et une fonction de perte MSE. Utiliser `evalMLMethod` prédire sur de nouvelles données. - -### Régression Logistique Stochastique {#stochastic-logistic-regression} - -Le [stochasticLogisticRegression](../../sql-reference/aggregate-functions/reference.md#agg_functions-stochasticlogisticregression) la fonction d'agrégation implémente la méthode de descente de gradient stochastique pour le problème de classification binaire. Utiliser `evalMLMethod` prédire sur de nouvelles données. diff --git a/docs/fr/sql-reference/functions/math-functions.md b/docs/fr/sql-reference/functions/math-functions.md deleted file mode 100644 index f5dff150caa..00000000000 --- a/docs/fr/sql-reference/functions/math-functions.md +++ /dev/null @@ -1,116 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: "Math\xE9matique" ---- - -# Fonctions Mathématiques {#mathematical-functions} - -Toutes les fonctions renvoient un nombre Float64. La précision du résultat est proche de la précision maximale possible, mais le résultat peut ne pas coïncider avec le nombre représentable de la machine le plus proche du nombre réel correspondant. - -## e() {#e} - -Renvoie un nombre Float64 proche du nombre E. - -## pi() {#pi} - -Returns a Float64 number that is close to the number π. - -## exp (x) {#expx} - -Accepte un argument numérique et renvoie un Float64 nombre proche de l'exposant de l'argument. - -## log(x), ln (x) {#logx-lnx} - -Accepte un argument numérique et renvoie un nombre Float64 proche du logarithme naturel de l'argument. - -## exp2 (x) {#exp2x} - -Accepte un argument numérique et renvoie un nombre Float64 proche de 2 à la puissance de X. - -## log2 (x) {#log2x} - -Accepte un argument numérique et renvoie un Float64 nombre proximité du logarithme binaire de l'argument. - -## exp10 (x) {#exp10x} - -Accepte un argument numérique et renvoie un nombre Float64 proche de 10 à la puissance de X. - -## log10 (x) {#log10x} - -Accepte un argument numérique et renvoie un nombre Float64 proche du logarithme décimal de l'argument. - -## sqrt (x) {#sqrtx} - -Accepte un argument numérique et renvoie un Float64 nombre proche de la racine carrée de l'argument. - -## cbrt (x) {#cbrtx} - -Accepte un argument numérique et renvoie un Float64 nombre proche de la racine cubique de l'argument. - -## erf (x) {#erfx} - -Si ‘x’ est non négatif, alors `erf(x / σ√2)` est la probabilité qu'une variable aléatoire ayant une distribution normale avec un écart type ‘σ’ prend la valeur qui est séparée de la valeur attendue par plus de ‘x’. - -Exemple (règle de trois sigma): - -``` sql -SELECT erf(3 / sqrt(2)) -``` - -``` text -┌─erf(divide(3, sqrt(2)))─┐ -│ 0.9973002039367398 │ -└─────────────────────────┘ -``` - -## erfc (x) {#erfcx} - -Accepte un argument numérique et renvoie un nombre Float64 proche de 1-erf (x), mais sans perte de précision pour ‘x’ valeur. - -## lgamma (x) {#lgammax} - -Le logarithme de la fonction gamma. - -## tgamma (x) {#tgammax} - -La fonction Gamma. - -## sin (x) {#sinx} - -Sine. - -## cos (x) {#cosx} - -Cosinus. - -## tan (x) {#tanx} - -Tangente. - -## asin (x) {#asinx} - -Le sinus d'arc. - -## acos (x) {#acosx} - -Le cosinus de l'arc. - -## atan (x) {#atanx} - -L'arc tangente. - -## pow(x, y), la puissance(x, y) {#powx-y-powerx-y} - -Prend deux arguments numériques x et Y. renvoie un nombre Float64 proche de x à la puissance de Y. - -## intExp2 {#intexp2} - -Accepte un argument numérique et renvoie un nombre UInt64 proche de 2 à la puissance de X. - -## intExp10 {#intexp10} - -Accepte un argument numérique et renvoie un nombre UInt64 proche de 10 à la puissance de X. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/math_functions/) diff --git a/docs/fr/sql-reference/functions/other-functions.md b/docs/fr/sql-reference/functions/other-functions.md deleted file mode 100644 index e5c6abedd75..00000000000 --- a/docs/fr/sql-reference/functions/other-functions.md +++ /dev/null @@ -1,1205 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 66 -toc_title: Autre ---- - -# D'Autres Fonctions {#other-functions} - -## hôte() {#hostname} - -Renvoie une chaîne avec le nom de l'hôte sur lequel cette fonction a été exécutée. Pour le traitement distribué, c'est le nom du serveur distant, si la fonction est exécutée sur un serveur distant. - -## getMacro {#getmacro} - -Obtient une valeur nommée à partir [macro](../../operations/server-configuration-parameters/settings.md#macros) la section de la configuration du serveur. - -**Syntaxe** - -``` sql -getMacro(name); -``` - -**Paramètre** - -- `name` — Name to retrieve from the `macros` section. [Chaîne](../../sql-reference/data-types/string.md#string). - -**Valeur renvoyée** - -- Valeur de la macro spécifiée. - -Type: [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -Exemple `macros` section dans le fichier de configuration du serveur: - -``` xml - - Value - -``` - -Requête: - -``` sql -SELECT getMacro('test'); -``` - -Résultat: - -``` text -┌─getMacro('test')─┐ -│ Value │ -└──────────────────┘ -``` - -Une méthode alternative pour obtenir la même valeur: - -``` sql -SELECT * FROM system.macros -WHERE macro = 'test'; -``` - -``` text -┌─macro─┬─substitution─┐ -│ test │ Value │ -└───────┴──────────────┘ -``` - -## FQDN {#fqdn} - -Retourne le nom de domaine pleinement qualifié. - -**Syntaxe** - -``` sql -fqdn(); -``` - -Cette fonction est insensible à la casse. - -**Valeur renvoyée** - -- Chaîne avec le nom de domaine complet. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT FQDN(); -``` - -Résultat: - -``` text -┌─FQDN()──────────────────────────┐ -│ clickhouse.ru-central1.internal │ -└─────────────────────────────────┘ -``` - -## basename {#basename} - -Extrait la partie finale d'une chaîne après la dernière barre oblique ou barre oblique inverse. Cette fonction est souvent utilisée pour extraire le nom de fichier d'un chemin. - -``` sql -basename( expr ) -``` - -**Paramètre** - -- `expr` — Expression resulting in a [Chaîne](../../sql-reference/data-types/string.md) type de valeur. Tous les antislashs doivent être échappés dans la valeur résultante. - -**Valeur Renvoyée** - -Une chaîne de caractères qui contient: - -- La partie finale d'une chaîne après la dernière barre oblique ou barre oblique inverse. - - If the input string contains a path ending with slash or backslash, for example, `/` or `c:\`, the function returns an empty string. - -- La chaîne d'origine s'il n'y a pas de barres obliques ou de barres obliques inverses. - -**Exemple** - -``` sql -SELECT 'some/long/path/to/file' AS a, basename(a) -``` - -``` text -┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ -│ some\long\path\to\file │ file │ -└────────────────────────┴────────────────────────────────────────┘ -``` - -``` sql -SELECT 'some\\long\\path\\to\\file' AS a, basename(a) -``` - -``` text -┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ -│ some\long\path\to\file │ file │ -└────────────────────────┴────────────────────────────────────────┘ -``` - -``` sql -SELECT 'some-file-name' AS a, basename(a) -``` - -``` text -┌─a──────────────┬─basename('some-file-name')─┐ -│ some-file-name │ some-file-name │ -└────────────────┴────────────────────────────┘ -``` - -## visibleWidth (x) {#visiblewidthx} - -Calcule la largeur approximative lors de la sortie des valeurs vers la console au format texte (séparé par des tabulations). -Cette fonction est utilisée par le système pour implémenter de jolis formats. - -`NULL` est représenté comme une chaîne correspondant à `NULL` dans `Pretty` format. - -``` sql -SELECT visibleWidth(NULL) -``` - -``` text -┌─visibleWidth(NULL)─┐ -│ 4 │ -└────────────────────┘ -``` - -## toTypeName (x) {#totypenamex} - -Renvoie une chaîne contenant le nom du type de l'argument passé. - -Si `NULL` est passé à la fonction en entrée, puis il renvoie le `Nullable(Nothing)` type, ce qui correspond à un interne `NULL` représentation à ClickHouse. - -## la taille de bloc() {#function-blocksize} - -Récupère la taille du bloc. -Dans ClickHouse, les requêtes sont toujours exécutées sur des blocs (ensembles de parties de colonne). Cette fonction permet d'obtenir la taille du bloc pour lequel vous l'avez appelé. - -## matérialiser (x) {#materializex} - -Transforme une constante dans une colonne contenant une seule valeur. -Dans ClickHouse, les colonnes complètes et les constantes sont représentées différemment en mémoire. Les fonctions fonctionnent différemment pour les arguments constants et les arguments normaux (un code différent est exécuté), bien que le résultat soit presque toujours le même. Cette fonction sert à déboguer ce comportement. - -## ignore(…) {#ignore} - -Accepte tous les arguments, y compris `NULL`. Renvoie toujours 0. -Cependant, l'argument est toujours évalué. Cela peut être utilisé pour les benchmarks. - -## sommeil(secondes) {#sleepseconds} - -Dormir ‘seconds’ secondes sur chaque bloc de données. Vous pouvez spécifier un nombre entier ou un nombre à virgule flottante. - -## sleepEachRow (secondes) {#sleepeachrowseconds} - -Dormir ‘seconds’ secondes sur chaque ligne. Vous pouvez spécifier un nombre entier ou un nombre à virgule flottante. - -## currentDatabase() {#currentdatabase} - -Retourne le nom de la base de données actuelle. -Vous pouvez utiliser cette fonction dans les paramètres du moteur de table dans une requête CREATE TABLE où vous devez spécifier la base de données. - -## currentUser() {#other-function-currentuser} - -Renvoie la connexion de l'utilisateur actuel. La connexion de l'utilisateur, cette requête initiée, sera renvoyée en cas de requête distibuted. - -``` sql -SELECT currentUser(); -``` - -Alias: `user()`, `USER()`. - -**Valeurs renvoyées** - -- Connexion de l'utilisateur actuel. -- Connexion de l'utilisateur qui a lancé la requête en cas de requête distribuée. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT currentUser(); -``` - -Résultat: - -``` text -┌─currentUser()─┐ -│ default │ -└───────────────┘ -``` - -## isConstant {#is-constant} - -Vérifie si l'argument est une expression constante. - -A constant expression means an expression whose resulting value is known at the query analysis (i.e. before execution). For example, expressions over [littéral](../syntax.md#literals) sont des expressions constantes. - -La fonction est destinée au développement, au débogage et à la démonstration. - -**Syntaxe** - -``` sql -isConstant(x) -``` - -**Paramètre** - -- `x` — Expression to check. - -**Valeurs renvoyées** - -- `1` — `x` est constante. -- `0` — `x` est non constante. - -Type: [UInt8](../data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT isConstant(x + 1) FROM (SELECT 43 AS x) -``` - -Résultat: - -``` text -┌─isConstant(plus(x, 1))─┐ -│ 1 │ -└────────────────────────┘ -``` - -Requête: - -``` sql -WITH 3.14 AS pi SELECT isConstant(cos(pi)) -``` - -Résultat: - -``` text -┌─isConstant(cos(pi))─┐ -│ 1 │ -└─────────────────────┘ -``` - -Requête: - -``` sql -SELECT isConstant(number) FROM numbers(1) -``` - -Résultat: - -``` text -┌─isConstant(number)─┐ -│ 0 │ -└────────────────────┘ -``` - -## isFinite (x) {#isfinitex} - -Accepte Float32 et Float64 et renvoie UInt8 égal à 1 si l'argument n'est pas infini et pas un NaN, sinon 0. - -## isInfinite (x) {#isinfinitex} - -Accepte Float32 et Float64 et renvoie UInt8 égal à 1 si l'argument est infini, sinon 0. Notez que 0 est retourné pour un NaN. - -## ifNotFinite {#ifnotfinite} - -Vérifie si la valeur à virgule flottante est finie. - -**Syntaxe** - - ifNotFinite(x,y) - -**Paramètre** - -- `x` — Value to be checked for infinity. Type: [Flottant\*](../../sql-reference/data-types/float.md). -- `y` — Fallback value. Type: [Flottant\*](../../sql-reference/data-types/float.md). - -**Valeur renvoyée** - -- `x` si `x` est finie. -- `y` si `x` n'est pas finie. - -**Exemple** - -Requête: - - SELECT 1/0 as infimum, ifNotFinite(infimum,42) - -Résultat: - - ┌─infimum─┬─ifNotFinite(divide(1, 0), 42)─┐ - │ inf │ 42 │ - └─────────┴───────────────────────────────┘ - -Vous pouvez obtenir un résultat similaire en utilisant [opérateur ternaire](conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. - -## isNaN (x) {#isnanx} - -Accepte Float32 et Float64 et renvoie UInt8 égal à 1 si l'argument est un NaN, sinon 0. - -## hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘database’, ‘table’, ‘column’) {#hascolumnintablehostname-username-password-database-table-column} - -Accepte les chaînes constantes: nom de la base de données, nom de la table et nom de la colonne. Renvoie une expression constante UInt8 égale à 1 s'il y a une colonne, sinon 0. Si le paramètre hostname est défini, le test s'exécutera sur un serveur distant. -La fonction renvoie une exception si la table n'existe pas. -Pour les éléments imbriqués structure des données, la fonction vérifie l'existence d'une colonne. Pour la structure de données imbriquée elle-même, la fonction renvoie 0. - -## bar {#function-bar} - -Permet de construire un diagramme unicode-art. - -`bar(x, min, max, width)` dessine une bande avec une largeur proportionnelle à `(x - min)` et égale à `width` les caractères lors de la `x = max`. - -Paramètre: - -- `x` — Size to display. -- `min, max` — Integer constants. The value must fit in `Int64`. -- `width` — Constant, positive integer, can be fractional. - -La bande dessinée avec précision à un huitième d'un symbole. - -Exemple: - -``` sql -SELECT - toHour(EventTime) AS h, - count() AS c, - bar(c, 0, 600000, 20) AS bar -FROM test.hits -GROUP BY h -ORDER BY h ASC -``` - -``` text -┌──h─┬──────c─┬─bar────────────────┐ -│ 0 │ 292907 │ █████████▋ │ -│ 1 │ 180563 │ ██████ │ -│ 2 │ 114861 │ ███▋ │ -│ 3 │ 85069 │ ██▋ │ -│ 4 │ 68543 │ ██▎ │ -│ 5 │ 78116 │ ██▌ │ -│ 6 │ 113474 │ ███▋ │ -│ 7 │ 170678 │ █████▋ │ -│ 8 │ 278380 │ █████████▎ │ -│ 9 │ 391053 │ █████████████ │ -│ 10 │ 457681 │ ███████████████▎ │ -│ 11 │ 493667 │ ████████████████▍ │ -│ 12 │ 509641 │ ████████████████▊ │ -│ 13 │ 522947 │ █████████████████▍ │ -│ 14 │ 539954 │ █████████████████▊ │ -│ 15 │ 528460 │ █████████████████▌ │ -│ 16 │ 539201 │ █████████████████▊ │ -│ 17 │ 523539 │ █████████████████▍ │ -│ 18 │ 506467 │ ████████████████▊ │ -│ 19 │ 520915 │ █████████████████▎ │ -│ 20 │ 521665 │ █████████████████▍ │ -│ 21 │ 542078 │ ██████████████████ │ -│ 22 │ 493642 │ ████████████████▍ │ -│ 23 │ 400397 │ █████████████▎ │ -└────┴────────┴────────────────────┘ -``` - -## transformer {#transform} - -Transforme une valeur en fonction explicitement définis cartographie de certains éléments à l'autre. -Il existe deux variantes de cette fonction: - -### de transformation(x, array_from, array_to, par défaut) {#transformx-array-from-array-to-default} - -`x` – What to transform. - -`array_from` – Constant array of values for converting. - -`array_to` – Constant array of values to convert the values in ‘from’ de. - -`default` – Which value to use if ‘x’ n'est pas égale à une des valeurs de ‘from’. - -`array_from` et `array_to` – Arrays of the same size. - -Type: - -`transform(T, Array(T), Array(U), U) -> U` - -`T` et `U` peuvent être des types numériques, chaîne ou Date ou DateTime. -Lorsque la même lettre est indiquée (T ou U), pour les types numériques, il se peut qu'il ne s'agisse pas de types correspondants, mais de types ayant un type commun. -Par exemple, le premier argument peut avoir le type Int64, tandis que le second a le type Array(UInt16). - -Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array_from’ tableau, elle renvoie l'élément existant (qui est numéroté de même) de la ‘array_to’ tableau. Sinon, elle renvoie ‘default’. S'il y a plusieurs éléments correspondants dans ‘array_from’ il renvoie l'un des matches. - -Exemple: - -``` sql -SELECT - transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other') AS title, - count() AS c -FROM test.hits -WHERE SearchEngineID != 0 -GROUP BY title -ORDER BY c DESC -``` - -``` text -┌─title─────┬──────c─┐ -│ Yandex │ 498635 │ -│ Google │ 229872 │ -│ Other │ 104472 │ -└───────────┴────────┘ -``` - -### de transformation(x, array_from, array_to) {#transformx-array-from-array-to} - -Diffère de la première variation en ce que le ‘default’ l'argument est omis. -Si l' ‘x’ la valeur est égale à l'un des éléments dans la ‘array_from’ tableau, elle renvoie l'élément correspondant (qui est numéroté de même) de la ‘array_to’ tableau. Sinon, elle renvoie ‘x’. - -Type: - -`transform(T, Array(T), Array(T)) -> T` - -Exemple: - -``` sql -SELECT - transform(domain(Referer), ['yandex.ru', 'google.ru', 'vk.com'], ['www.yandex', 'example.com']) AS s, - count() AS c -FROM test.hits -GROUP BY domain(Referer) -ORDER BY count() DESC -LIMIT 10 -``` - -``` text -┌─s──────────────┬───────c─┐ -│ │ 2906259 │ -│ www.yandex │ 867767 │ -│ ███████.ru │ 313599 │ -│ mail.yandex.ru │ 107147 │ -│ ██████.ru │ 100355 │ -│ █████████.ru │ 65040 │ -│ news.yandex.ru │ 64515 │ -│ ██████.net │ 59141 │ -│ example.com │ 57316 │ -└────────────────┴─────────┘ -``` - -## formatReadableSize (x) {#formatreadablesizex} - -Accepte la taille (nombre d'octets). Renvoie une taille arrondie avec un suffixe (KiB, MiB, etc.) comme une chaîne de caractères. - -Exemple: - -``` sql -SELECT - arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, - formatReadableSize(filesize_bytes) AS filesize -``` - -``` text -┌─filesize_bytes─┬─filesize───┐ -│ 1 │ 1.00 B │ -│ 1024 │ 1.00 KiB │ -│ 1048576 │ 1.00 MiB │ -│ 192851925 │ 183.92 MiB │ -└────────────────┴────────────┘ -``` - -## moins (a, b) {#leasta-b} - -Renvoie la plus petite valeur de a et b. - -## la plus grande(a, b) {#greatesta-b} - -Renvoie la plus grande valeur de a et B. - -## le temps de disponibilité() {#uptime} - -Renvoie la disponibilité du serveur en quelques secondes. - -## version() {#version} - -Renvoie la version du serveur sous forme de chaîne. - -## fuseau() {#timezone} - -Retourne le fuseau horaire du serveur. - -## blockNumber {#blocknumber} - -Renvoie le numéro de séquence du bloc de données où se trouve la ligne. - -## rowNumberInBlock {#function-rownumberinblock} - -Renvoie le numéro de séquence de la ligne dans le bloc de données. Différents blocs de données sont toujours recalculés. - -## rowNumberInAllBlocks() {#rownumberinallblocks} - -Renvoie le numéro de séquence de la ligne dans le bloc de données. Cette fonction ne prend en compte que les blocs de données affectés. - -## voisin {#neighbor} - -La fonction de fenêtre qui donne accès à une ligne à un décalage spécifié qui vient avant ou après la ligne actuelle d'une colonne donnée. - -**Syntaxe** - -``` sql -neighbor(column, offset[, default_value]) -``` - -Le résultat de la fonction dépend du touché des blocs de données et l'ordre des données dans le bloc. -Si vous créez une sous-requête avec ORDER BY et appelez la fonction depuis l'extérieur de la sous-requête, vous pouvez obtenir le résultat attendu. - -**Paramètre** - -- `column` — A column name or scalar expression. -- `offset` — The number of rows forwards or backwards from the current row of `column`. [Int64](../../sql-reference/data-types/int-uint.md). -- `default_value` — Optional. The value to be returned if offset goes beyond the scope of the block. Type of data blocks affected. - -**Valeurs renvoyées** - -- De la valeur pour `column` dans `offset` distance de la ligne actuelle si `offset` la valeur n'est pas en dehors des limites du bloc. -- La valeur par défaut pour `column` si `offset` la valeur est en dehors des limites du bloc. Si `default_value` est donné, alors il sera utilisé. - -Type: type de blocs de données affectés ou type de valeur par défaut. - -**Exemple** - -Requête: - -``` sql -SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; -``` - -Résultat: - -``` text -┌─number─┬─neighbor(number, 2)─┐ -│ 0 │ 2 │ -│ 1 │ 3 │ -│ 2 │ 4 │ -│ 3 │ 5 │ -│ 4 │ 6 │ -│ 5 │ 7 │ -│ 6 │ 8 │ -│ 7 │ 9 │ -│ 8 │ 0 │ -│ 9 │ 0 │ -└────────┴─────────────────────┘ -``` - -Requête: - -``` sql -SELECT number, neighbor(number, 2, 999) FROM system.numbers LIMIT 10; -``` - -Résultat: - -``` text -┌─number─┬─neighbor(number, 2, 999)─┐ -│ 0 │ 2 │ -│ 1 │ 3 │ -│ 2 │ 4 │ -│ 3 │ 5 │ -│ 4 │ 6 │ -│ 5 │ 7 │ -│ 6 │ 8 │ -│ 7 │ 9 │ -│ 8 │ 999 │ -│ 9 │ 999 │ -└────────┴──────────────────────────┘ -``` - -Cette fonction peut être utilisée pour calculer une année à valeur métrique: - -Requête: - -``` sql -WITH toDate('2018-01-01') AS start_date -SELECT - toStartOfMonth(start_date + (number * 32)) AS month, - toInt32(month) % 100 AS money, - neighbor(money, -12) AS prev_year, - round(prev_year / money, 2) AS year_over_year -FROM numbers(16) -``` - -Résultat: - -``` text -┌──────month─┬─money─┬─prev_year─┬─year_over_year─┐ -│ 2018-01-01 │ 32 │ 0 │ 0 │ -│ 2018-02-01 │ 63 │ 0 │ 0 │ -│ 2018-03-01 │ 91 │ 0 │ 0 │ -│ 2018-04-01 │ 22 │ 0 │ 0 │ -│ 2018-05-01 │ 52 │ 0 │ 0 │ -│ 2018-06-01 │ 83 │ 0 │ 0 │ -│ 2018-07-01 │ 13 │ 0 │ 0 │ -│ 2018-08-01 │ 44 │ 0 │ 0 │ -│ 2018-09-01 │ 75 │ 0 │ 0 │ -│ 2018-10-01 │ 5 │ 0 │ 0 │ -│ 2018-11-01 │ 36 │ 0 │ 0 │ -│ 2018-12-01 │ 66 │ 0 │ 0 │ -│ 2019-01-01 │ 97 │ 32 │ 0.33 │ -│ 2019-02-01 │ 28 │ 63 │ 2.25 │ -│ 2019-03-01 │ 56 │ 91 │ 1.62 │ -│ 2019-04-01 │ 87 │ 22 │ 0.25 │ -└────────────┴───────┴───────────┴────────────────┘ -``` - -## runningDifference(x) {#other_functions-runningdifference} - -Calculates the difference between successive row values ​​in the data block. -Renvoie 0 pour la première ligne et la différence par rapport à la rangée précédente pour chaque nouvelle ligne. - -Le résultat de la fonction dépend du touché des blocs de données et l'ordre des données dans le bloc. -Si vous créez une sous-requête avec ORDER BY et appelez la fonction depuis l'extérieur de la sous-requête, vous pouvez obtenir le résultat attendu. - -Exemple: - -``` sql -SELECT - EventID, - EventTime, - runningDifference(EventTime) AS delta -FROM -( - SELECT - EventID, - EventTime - FROM events - WHERE EventDate = '2016-11-24' - ORDER BY EventTime ASC - LIMIT 5 -) -``` - -``` text -┌─EventID─┬───────────EventTime─┬─delta─┐ -│ 1106 │ 2016-11-24 00:00:04 │ 0 │ -│ 1107 │ 2016-11-24 00:00:05 │ 1 │ -│ 1108 │ 2016-11-24 00:00:05 │ 0 │ -│ 1109 │ 2016-11-24 00:00:09 │ 4 │ -│ 1110 │ 2016-11-24 00:00:10 │ 1 │ -└─────────┴─────────────────────┴───────┘ -``` - -Veuillez noter que la taille du bloc affecte le résultat. Avec chaque nouveau bloc, le `runningDifference` l'état est réinitialisé. - -``` sql -SELECT - number, - runningDifference(number + 1) AS diff -FROM numbers(100000) -WHERE diff != 1 -``` - -``` text -┌─number─┬─diff─┐ -│ 0 │ 0 │ -└────────┴──────┘ -┌─number─┬─diff─┐ -│ 65536 │ 0 │ -└────────┴──────┘ -``` - -``` sql -set max_block_size=100000 -- default value is 65536! - -SELECT - number, - runningDifference(number + 1) AS diff -FROM numbers(100000) -WHERE diff != 1 -``` - -``` text -┌─number─┬─diff─┐ -│ 0 │ 0 │ -└────────┴──────┘ -``` - -## runningDifferenceStartingWithFirstvalue {#runningdifferencestartingwithfirstvalue} - -De même que pour [runningDifference](./other-functions.md#other_functions-runningdifference) la différence est la valeur de la première ligne, est retourné à la valeur de la première ligne, et chaque rangée suivante renvoie la différence de la rangée précédente. - -## MACNumToString (num) {#macnumtostringnum} - -Accepte un numéro UInt64. Interprète comme une adresse MAC dans big endian. Renvoie une chaîne contenant l'adresse MAC correspondante au format AA:BB:CC: DD:EE: FF (Nombres séparés par deux points sous forme hexadécimale). - -## MACStringToNum (s) {#macstringtonums} - -La fonction inverse de MACNumToString. Si l'adresse MAC a un format non valide, elle renvoie 0. - -## MACStringToOUI (s) {#macstringtoouis} - -Accepte une adresse MAC au format AA:BB:CC: DD:EE: FF (Nombres séparés par deux points sous forme hexadécimale). Renvoie les trois premiers octets sous la forme D'un nombre UInt64. Si l'adresse MAC a un format non valide, elle renvoie 0. - -## getSizeOfEnumType {#getsizeofenumtype} - -Retourne le nombre de champs dans [Enum](../../sql-reference/data-types/enum.md). - -``` sql -getSizeOfEnumType(value) -``` - -**Paramètre:** - -- `value` — Value of type `Enum`. - -**Valeurs renvoyées** - -- Le nombre de champs avec `Enum` les valeurs d'entrée. -- Une exception est levée si le type n'est pas `Enum`. - -**Exemple** - -``` sql -SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x -``` - -``` text -┌─x─┐ -│ 2 │ -└───┘ -``` - -## blockSerializedSize {#blockserializedsize} - -Retourne la taille sur le disque (sans tenir compte de la compression). - -``` sql -blockSerializedSize(value[, value[, ...]]) -``` - -**Paramètre:** - -- `value` — Any value. - -**Valeurs renvoyées** - -- Le nombre d'octets qui seront écrites sur le disque pour le bloc de valeurs (sans compression). - -**Exemple** - -``` sql -SELECT blockSerializedSize(maxState(1)) as x -``` - -``` text -┌─x─┐ -│ 2 │ -└───┘ -``` - -## toColumnTypeName {#tocolumntypename} - -Renvoie le nom de la classe qui représente le type de données de la colonne dans la RAM. - -``` sql -toColumnTypeName(value) -``` - -**Paramètre:** - -- `value` — Any type of value. - -**Valeurs renvoyées** - -- Une chaîne avec le nom de la classe utilisée pour représenter `value` type de données dans la mémoire RAM. - -**Exemple de la différence entre`toTypeName ' and ' toColumnTypeName`** - -``` sql -SELECT toTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) -``` - -``` text -┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ -│ DateTime │ -└─────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT toColumnTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) -``` - -``` text -┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ -│ Const(UInt32) │ -└───────────────────────────────────────────────────────────┘ -``` - -L'exemple montre que le `DateTime` type de données est stocké dans la mémoire comme `Const(UInt32)`. - -## dumpColumnStructure {#dumpcolumnstructure} - -Affiche une description détaillée des structures de données en RAM - -``` sql -dumpColumnStructure(value) -``` - -**Paramètre:** - -- `value` — Any type of value. - -**Valeurs renvoyées** - -- Une chaîne décrivant la structure utilisée pour représenter `value` type de données dans la mémoire RAM. - -**Exemple** - -``` sql -SELECT dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime')) -``` - -``` text -┌─dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ -│ DateTime, Const(size = 1, UInt32(size = 1)) │ -└──────────────────────────────────────────────────────────────┘ -``` - -## defaultValueOfArgumentType {#defaultvalueofargumenttype} - -Affiche la valeur par défaut du type de données. - -Ne pas inclure des valeurs par défaut pour les colonnes personnalisées définies par l'utilisateur. - -``` sql -defaultValueOfArgumentType(expression) -``` - -**Paramètre:** - -- `expression` — Arbitrary type of value or an expression that results in a value of an arbitrary type. - -**Valeurs renvoyées** - -- `0` pour les nombres. -- Chaîne vide pour les chaînes. -- `ᴺᵁᴸᴸ` pour [Nullable](../../sql-reference/data-types/nullable.md). - -**Exemple** - -``` sql -SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) -``` - -``` text -┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ -│ 0 │ -└─────────────────────────────────────────────┘ -``` - -``` sql -SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) -``` - -``` text -┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ -│ ᴺᵁᴸᴸ │ -└───────────────────────────────────────────────────────┘ -``` - -## reproduire {#other-functions-replicate} - -Crée un tableau avec une seule valeur. - -Utilisé pour la mise en œuvre interne de [arrayJoin](array-join.md#functions_arrayjoin). - -``` sql -SELECT replicate(x, arr); -``` - -**Paramètre:** - -- `arr` — Original array. ClickHouse creates a new array of the same length as the original and fills it with the value `x`. -- `x` — The value that the resulting array will be filled with. - -**Valeur renvoyée** - -Un tableau rempli de la valeur `x`. - -Type: `Array`. - -**Exemple** - -Requête: - -``` sql -SELECT replicate(1, ['a', 'b', 'c']) -``` - -Résultat: - -``` text -┌─replicate(1, ['a', 'b', 'c'])─┐ -│ [1,1,1] │ -└───────────────────────────────┘ -``` - -## filesystemAvailable {#filesystemavailable} - -Renvoie la quantité d'espace restant sur le système de fichiers où se trouvent les fichiers des bases de données. Il est toujours plus petit que l'espace libre total ([filesystemFree](#filesystemfree)) parce qu'un peu d'espace est réservé au système D'exploitation. - -**Syntaxe** - -``` sql -filesystemAvailable() -``` - -**Valeur renvoyée** - -- La quantité d'espace restant disponible en octets. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT formatReadableSize(filesystemAvailable()) AS "Available space", toTypeName(filesystemAvailable()) AS "Type"; -``` - -Résultat: - -``` text -┌─Available space─┬─Type───┐ -│ 30.75 GiB │ UInt64 │ -└─────────────────┴────────┘ -``` - -## filesystemFree {#filesystemfree} - -Retourne montant total de l'espace libre sur le système de fichiers où les fichiers des bases de données. Voir aussi `filesystemAvailable` - -**Syntaxe** - -``` sql -filesystemFree() -``` - -**Valeur renvoyée** - -- Quantité d'espace libre en octets. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT formatReadableSize(filesystemFree()) AS "Free space", toTypeName(filesystemFree()) AS "Type"; -``` - -Résultat: - -``` text -┌─Free space─┬─Type───┐ -│ 32.39 GiB │ UInt64 │ -└────────────┴────────┘ -``` - -## filesystemCapacity {#filesystemcapacity} - -Renvoie la capacité du système de fichiers en octets. Pour l'évaluation, la [chemin](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) le répertoire de données doit être configuré. - -**Syntaxe** - -``` sql -filesystemCapacity() -``` - -**Valeur renvoyée** - -- Informations de capacité du système de fichiers en octets. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT formatReadableSize(filesystemCapacity()) AS "Capacity", toTypeName(filesystemCapacity()) AS "Type" -``` - -Résultat: - -``` text -┌─Capacity──┬─Type───┐ -│ 39.32 GiB │ UInt64 │ -└───────────┴────────┘ -``` - -## finalizeAggregation {#function-finalizeaggregation} - -Prend de l'état de la fonction d'agrégation. Renvoie le résultat de l'agrégation (état finalisé). - -## runningAccumulate {#function-runningaccumulate} - -Prend les membres de la fonction d'agrégation et renvoie une colonne avec des valeurs, sont le résultat de l'accumulation de ces états pour un ensemble de bloc de lignes, de la première à la ligne actuelle. -Par exemple, prend l'état de la fonction d'agrégat (exemple runningAccumulate(uniqState(UserID))), et pour chaque ligne de bloc, retourne le résultat de la fonction d'agrégat lors de la fusion des états de toutes les lignes précédentes et de la ligne actuelle. -Ainsi, le résultat de la fonction dépend de la partition des données aux blocs et de l'ordre des données dans le bloc. - -## joinGet {#joinget} - -La fonction vous permet d'extraire les données de la table de la même manière qu'à partir d'un [dictionnaire](../../sql-reference/dictionaries/index.md). - -Obtient les données de [Rejoindre](../../engines/table-engines/special/join.md#creating-a-table) tables utilisant la clé de jointure spécifiée. - -Ne prend en charge que les tables créées avec `ENGINE = Join(ANY, LEFT, )` déclaration. - -**Syntaxe** - -``` sql -joinGet(join_storage_table_name, `value_column`, join_keys) -``` - -**Paramètre** - -- `join_storage_table_name` — an [identificateur](../syntax.md#syntax-identifiers) indique l'endroit où la recherche est effectuée. L'identificateur est recherché dans la base de données par défaut (voir paramètre `default_database` dans le fichier de config). Pour remplacer la base de données par défaut, utilisez `USE db_name` ou spécifiez la base de données et la table via le séparateur `db_name.db_table` voir l'exemple. -- `value_column` — name of the column of the table that contains required data. -- `join_keys` — list of keys. - -**Valeur renvoyée** - -Retourne la liste des valeurs correspond à la liste des clés. - -Si certain n'existe pas dans la table source alors `0` ou `null` seront renvoyés basé sur [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) paramètre. - -Plus d'infos sur `join_use_nulls` dans [Opération de jointure](../../engines/table-engines/special/join.md). - -**Exemple** - -Table d'entrée: - -``` sql -CREATE DATABASE db_test -CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1 -INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) -``` - -``` text -┌─id─┬─val─┐ -│ 4 │ 13 │ -│ 2 │ 12 │ -│ 1 │ 11 │ -└────┴─────┘ -``` - -Requête: - -``` sql -SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 -``` - -Résultat: - -``` text -┌─joinGet(db_test.id_val, 'val', toUInt32(number))─┐ -│ 0 │ -│ 11 │ -│ 12 │ -│ 0 │ -└──────────────────────────────────────────────────┘ -``` - -## modelEvaluate(model_name, …) {#function-modelevaluate} - -Évaluer le modèle externe. -Accepte un nom de modèle et le modèle de l'argumentation. Renvoie Float64. - -## throwIf (x \[, custom_message\]) {#throwifx-custom-message} - -Lever une exception si l'argument est non nul. -custom_message - est un paramètre optionnel: une chaîne constante, fournit un message d'erreur - -``` sql -SELECT throwIf(number = 3, 'Too many') FROM numbers(10); -``` - -``` text -↙ Progress: 0.00 rows, 0.00 B (0.00 rows/s., 0.00 B/s.) Received exception from server (version 19.14.1): -Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. -``` - -## identité {#identity} - -Renvoie la même valeur qui a été utilisée comme argument. Utilisé pour le débogage et les tests, permet d'annuler l'utilisation de l'index et d'obtenir les performances de requête d'une analyse complète. Lorsque la requête est analysée pour une utilisation possible de l'index, l'analyseur ne regarde pas à l'intérieur `identity` fonction. - -**Syntaxe** - -``` sql -identity(x) -``` - -**Exemple** - -Requête: - -``` sql -SELECT identity(42) -``` - -Résultat: - -``` text -┌─identity(42)─┐ -│ 42 │ -└──────────────┘ -``` - -## randomPrintableASCII {#randomascii} - -Génère une chaîne avec un ensemble aléatoire de [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) caractères imprimables. - -**Syntaxe** - -``` sql -randomPrintableASCII(length) -``` - -**Paramètre** - -- `length` — Resulting string length. Positive integer. - - If you pass `length < 0`, behavior of the function is undefined. - -**Valeur renvoyée** - -- Chaîne avec un ensemble aléatoire de [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) caractères imprimables. - -Type: [Chaîne](../../sql-reference/data-types/string.md) - -**Exemple** - -``` sql -SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 -``` - -``` text -┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ -│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ -│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ -│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ -└────────┴────────────────────────────────┴──────────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/other_functions/) diff --git a/docs/fr/sql-reference/functions/random-functions.md b/docs/fr/sql-reference/functions/random-functions.md deleted file mode 100644 index 3c4e15507bb..00000000000 --- a/docs/fr/sql-reference/functions/random-functions.md +++ /dev/null @@ -1,65 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 51 -toc_title: "La G\xE9n\xE9ration De Nombres Pseudo-Al\xE9atoires" ---- - -# Fonctions pour générer des nombres Pseudo-aléatoires {#functions-for-generating-pseudo-random-numbers} - -Des générateurs Non cryptographiques de nombres pseudo-aléatoires sont utilisés. - -Toutes les fonctions acceptent zéro argument ou un argument. -Si un argument est passé, il peut être de n'importe quel type, et sa valeur n'est utilisée pour rien. -Le seul but de cet argument est d'empêcher l'élimination des sous-expressions courantes, de sorte que deux instances différentes de la même fonction renvoient des colonnes différentes avec des nombres aléatoires différents. - -## Rand {#rand} - -Renvoie un nombre UInt32 pseudo-aléatoire, réparti uniformément entre tous les nombres de type UInt32. -Utilise un générateur congruentiel linéaire. - -## rand64 {#rand64} - -Renvoie un nombre UInt64 pseudo-aléatoire, réparti uniformément entre tous les nombres de type UInt64. -Utilise un générateur congruentiel linéaire. - -## randConstant {#randconstant} - -Produit une colonne constante avec une valeur aléatoire. - -**Syntaxe** - -``` sql -randConstant([x]) -``` - -**Paramètre** - -- `x` — [Expression](../syntax.md#syntax-expressions) résultant de la [types de données pris en charge](../data-types/index.md#data_types). La valeur résultante est ignorée, mais l'expression elle-même si elle est utilisée pour contourner [élimination des sous-expressions courantes](index.md#common-subexpression-elimination) si la fonction est appelée plusieurs fois dans une seule requête. Paramètre facultatif. - -**Valeur renvoyée** - -- Nombre Pseudo-aléatoire. - -Type: [UInt32](../data-types/int-uint.md). - -**Exemple** - -Requête: - -``` sql -SELECT rand(), rand(1), rand(number), randConstant(), randConstant(1), randConstant(number) -FROM numbers(3) -``` - -Résultat: - -``` text -┌─────rand()─┬────rand(1)─┬─rand(number)─┬─randConstant()─┬─randConstant(1)─┬─randConstant(number)─┐ -│ 3047369878 │ 4132449925 │ 4044508545 │ 2740811946 │ 4229401477 │ 1924032898 │ -│ 2938880146 │ 1267722397 │ 4154983056 │ 2740811946 │ 4229401477 │ 1924032898 │ -│ 956619638 │ 4238287282 │ 1104342490 │ 2740811946 │ 4229401477 │ 1924032898 │ -└────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/random_functions/) diff --git a/docs/fr/sql-reference/functions/rounding-functions.md b/docs/fr/sql-reference/functions/rounding-functions.md deleted file mode 100644 index f99e6358026..00000000000 --- a/docs/fr/sql-reference/functions/rounding-functions.md +++ /dev/null @@ -1,190 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: Arrondi ---- - -# Fonctions D'Arrondi {#rounding-functions} - -## floor(x\[, N\]) {#floorx-n} - -Renvoie le plus grand nombre rond inférieur ou égal à `x`. Un nombre rond est un multiple de 1 / 10N, ou le nombre le plus proche du type de données approprié si 1 / 10N n'est pas exact. -‘N’ est une constante entière, paramètre facultatif. Par défaut, il est zéro, ce qui signifie arrondir à un entier. -‘N’ peut être négative. - -Exemple: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.` - -`x` est n'importe quel type numérique. Le résultat est un nombre du même type. -Pour les arguments entiers, il est logique d'arrondir avec un négatif `N` valeur (pour non négatif `N`, la fonction ne fait rien). -Si l'arrondi provoque un débordement (par exemple, floor(-128, -1)), un résultat spécifique à l'implémentation est renvoyé. - -## ceil(x\[, n\]), plafond (x\[, n\]) {#ceilx-n-ceilingx-n} - -Renvoie le plus petit nombre rond supérieur ou égal à `x`. Dans tous les autres sens, il est le même que le `floor` fonction (voir ci-dessus). - -## trunc(x \[, N\]), truncate(x \[, N\]) {#truncx-n-truncatex-n} - -Renvoie le nombre rond avec la plus grande valeur absolue qui a une valeur absolue inférieure ou égale à `x`‘s. In every other way, it is the same as the ’floor’ fonction (voir ci-dessus). - -## round(x\[, N\]) {#rounding_functions-round} - -Arrondit une valeur à un nombre spécifié de décimales. - -La fonction renvoie le nombre plus proche de l'ordre spécifié. Dans le cas où un nombre donné a une distance égale aux nombres environnants, la fonction utilise l'arrondi de banquier pour les types de nombres flottants et arrondit à partir de zéro pour les autres types de nombres. - -``` sql -round(expression [, decimal_places]) -``` - -**Paramètre:** - -- `expression` — A number to be rounded. Can be any [expression](../syntax.md#syntax-expressions) retour du numérique [type de données](../../sql-reference/data-types/index.md#data_types). -- `decimal-places` — An integer value. - - Si `decimal-places > 0` alors la fonction arrondit la valeur à droite du point décimal. - - Si `decimal-places < 0` alors la fonction arrondit la valeur à gauche de la virgule décimale. - - Si `decimal-places = 0` alors la fonction arrondit la valeur à l'entier. Dans ce cas, l'argument peut être omis. - -**Valeur renvoyée:** - -Le nombre arrondi du même type que le nombre d'entrée. - -### Exemple {#examples} - -**Exemple d'utilisation** - -``` sql -SELECT number / 2 AS x, round(x) FROM system.numbers LIMIT 3 -``` - -``` text -┌───x─┬─round(divide(number, 2))─┐ -│ 0 │ 0 │ -│ 0.5 │ 0 │ -│ 1 │ 1 │ -└─────┴──────────────────────────┘ -``` - -**Des exemples de l'arrondissement** - -Le résultat est arrondi au plus proche. - -``` text -round(3.2, 0) = 3 -round(4.1267, 2) = 4.13 -round(22,-1) = 20 -round(467,-2) = 500 -round(-467,-2) = -500 -``` - -Le Banquier arrondit. - -``` text -round(3.5) = 4 -round(4.5) = 4 -round(3.55, 1) = 3.6 -round(3.65, 1) = 3.6 -``` - -**Voir Aussi** - -- [roundBankers](#roundbankers) - -## roundBankers {#roundbankers} - -Arrondit un nombre à une position décimale spécifiée. - -- Si le nombre est arrondi à mi-chemin entre deux nombres, la fonction utilise l'arrondi. - - Banker's rounding is a method of rounding fractional numbers. When the rounding number is halfway between two numbers, it's rounded to the nearest even digit at the specified decimal position. For example: 3.5 rounds up to 4, 2.5 rounds down to 2. - - It's the default rounding method for floating point numbers defined in [IEEE 754](https://en.wikipedia.org/wiki/IEEE_754#Roundings_to_nearest). The [round](#rounding_functions-round) function performs the same rounding for floating point numbers. The `roundBankers` function also rounds integers the same way, for example, `roundBankers(45, -1) = 40`. - -- Dans d'autres cas, la fonction arrondit les nombres à l'entier le plus proche. - -À l'aide de l'arrondi, vous pouvez réduire l'effet qu'arrondir les nombres sur les résultats d'additionner ou de soustraire ces chiffres. - -Par exemple, les nombres de somme 1.5, 2.5, 3.5, 4.5 avec des arrondis différents: - -- Pas d'arrondi: 1.5 + 2.5 + 3.5 + 4.5 = 12. -- Arrondi du banquier: 2 + 2 + 4 + 4 = 12. -- Arrondi à l'entier le plus proche: 2 + 3 + 4 + 5 = 14. - -**Syntaxe** - -``` sql -roundBankers(expression [, decimal_places]) -``` - -**Paramètre** - -- `expression` — A number to be rounded. Can be any [expression](../syntax.md#syntax-expressions) retour du numérique [type de données](../../sql-reference/data-types/index.md#data_types). -- `decimal-places` — Decimal places. An integer number. - - `decimal-places > 0` — The function rounds the number to the given position right of the decimal point. Example: `roundBankers(3.55, 1) = 3.6`. - - `decimal-places < 0` — The function rounds the number to the given position left of the decimal point. Example: `roundBankers(24.55, -1) = 20`. - - `decimal-places = 0` — The function rounds the number to an integer. In this case the argument can be omitted. Example: `roundBankers(2.5) = 2`. - -**Valeur renvoyée** - -Valeur arrondie par la méthode d'arrondi du banquier. - -### Exemple {#examples-1} - -**Exemple d'utilisation** - -Requête: - -``` sql - SELECT number / 2 AS x, roundBankers(x, 0) AS b fROM system.numbers limit 10 -``` - -Résultat: - -``` text -┌───x─┬─b─┐ -│ 0 │ 0 │ -│ 0.5 │ 0 │ -│ 1 │ 1 │ -│ 1.5 │ 2 │ -│ 2 │ 2 │ -│ 2.5 │ 2 │ -│ 3 │ 3 │ -│ 3.5 │ 4 │ -│ 4 │ 4 │ -│ 4.5 │ 4 │ -└─────┴───┘ -``` - -**Exemples d'arrondi bancaire** - -``` text -roundBankers(0.4) = 0 -roundBankers(-3.5) = -4 -roundBankers(4.5) = 4 -roundBankers(3.55, 1) = 3.6 -roundBankers(3.65, 1) = 3.6 -roundBankers(10.35, 1) = 10.4 -roundBankers(10.755, 2) = 11,76 -``` - -**Voir Aussi** - -- [rond](#rounding_functions-round) - -## roundToExp2 (num) {#roundtoexp2num} - -Accepte un certain nombre. Si le nombre est inférieur à un, elle renvoie 0. Sinon, il arrondit le nombre au degré le plus proche (entier non négatif) de deux. - -## roundDuration (num) {#rounddurationnum} - -Accepte un certain nombre. Si le nombre est inférieur à un, elle renvoie 0. Sinon, il arrondit le nombre vers le bas pour les nombres de l'ensemble: 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000. Cette fonction est spécifique à Yandex.Metrica et utilisé pour la mise en œuvre du rapport sur la durée de la session. - -## roundAge (num) {#roundagenum} - -Accepte un certain nombre. Si le nombre est inférieur à 18, il renvoie 0. Sinon, il arrondit le nombre à un nombre de l'ensemble: 18, 25, 35, 45, 55. Cette fonction est spécifique à Yandex.Metrica et utilisé pour la mise en œuvre du rapport sur l'âge des utilisateurs. - -## roundDown(num, arr) {#rounddownnum-arr} - -Accepte un nombre et l'arrondit à un élément dans le tableau spécifié. Si la valeur est inférieure à la plus basse, la plus basse lié est retourné. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/rounding_functions/) diff --git a/docs/fr/sql-reference/functions/splitting-merging-functions.md b/docs/fr/sql-reference/functions/splitting-merging-functions.md deleted file mode 100644 index a1260e918b0..00000000000 --- a/docs/fr/sql-reference/functions/splitting-merging-functions.md +++ /dev/null @@ -1,116 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 47 -toc_title: "Fractionnement et fusion de cha\xEEnes et de tableaux" ---- - -# Fonctions pour diviser et fusionner des chaînes et des tableaux {#functions-for-splitting-and-merging-strings-and-arrays} - -## splitByChar (séparateur, s) {#splitbycharseparator-s} - -Divise une chaîne en sous-chaînes séparées par un caractère spécifique. Il utilise une chaîne constante `separator` qui composé d'un seul caractère. -Retourne un tableau de certaines chaînes. Les sous-chaînes vides peuvent être sélectionnées si le séparateur se produit au début ou à la fin de la chaîne, ou s'il existe plusieurs séparateurs consécutifs. - -**Syntaxe** - -``` sql -splitByChar(, ) -``` - -**Paramètre** - -- `separator` — The separator which should contain exactly one character. [Chaîne](../../sql-reference/data-types/string.md). -- `s` — The string to split. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée(s)** - -Retourne un tableau de certaines chaînes. Des sous-chaînes vides peuvent être sélectionnées lorsque: - -- Un séparateur se produit au début ou à la fin de la chaîne; -- Il existe plusieurs séparateurs consécutifs; -- La chaîne d'origine `s` est vide. - -Type: [Tableau](../../sql-reference/data-types/array.md) de [Chaîne](../../sql-reference/data-types/string.md). - -**Exemple** - -``` sql -SELECT splitByChar(',', '1,2,3,abcde') -``` - -``` text -┌─splitByChar(',', '1,2,3,abcde')─┐ -│ ['1','2','3','abcde'] │ -└─────────────────────────────────┘ -``` - -## splitByString(séparateur, s) {#splitbystringseparator-s} - -Divise une chaîne en sous-chaînes séparées par une chaîne. Il utilise une chaîne constante `separator` de plusieurs caractères comme séparateur. Si la chaîne `separator` est vide, il va diviser la chaîne `s` dans un tableau de caractères uniques. - -**Syntaxe** - -``` sql -splitByString(, ) -``` - -**Paramètre** - -- `separator` — The separator. [Chaîne](../../sql-reference/data-types/string.md). -- `s` — The string to split. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée(s)** - -Retourne un tableau de certaines chaînes. Des sous-chaînes vides peuvent être sélectionnées lorsque: - -Type: [Tableau](../../sql-reference/data-types/array.md) de [Chaîne](../../sql-reference/data-types/string.md). - -- Un séparateur non vide se produit au début ou à la fin de la chaîne; -- Il existe plusieurs séparateurs consécutifs non vides; -- La chaîne d'origine `s` est vide tandis que le séparateur n'est pas vide. - -**Exemple** - -``` sql -SELECT splitByString(', ', '1, 2 3, 4,5, abcde') -``` - -``` text -┌─splitByString(', ', '1, 2 3, 4,5, abcde')─┐ -│ ['1','2 3','4,5','abcde'] │ -└───────────────────────────────────────────┘ -``` - -``` sql -SELECT splitByString('', 'abcde') -``` - -``` text -┌─splitByString('', 'abcde')─┐ -│ ['a','b','c','d','e'] │ -└────────────────────────────┘ -``` - -## arrayStringConcat(arr \[, séparateur\]) {#arraystringconcatarr-separator} - -Concatène les chaînes répertoriées dans le tableau avec le séparateur."séparateur" est un paramètre facultatif: une chaîne constante, définie à une chaîne vide par défaut. -Retourne une chaîne de caractères. - -## alphaTokens (s) {#alphatokenss} - -Sélectionne des sous-chaînes d'octets consécutifs dans les plages A-z et A-Z. retourne un tableau de sous-chaînes. - -**Exemple** - -``` sql -SELECT alphaTokens('abca1abc') -``` - -``` text -┌─alphaTokens('abca1abc')─┐ -│ ['abca','abc'] │ -└─────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/splitting_merging_functions/) diff --git a/docs/fr/sql-reference/functions/string-functions.md b/docs/fr/sql-reference/functions/string-functions.md deleted file mode 100644 index 1482952426c..00000000000 --- a/docs/fr/sql-reference/functions/string-functions.md +++ /dev/null @@ -1,489 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: "Travailler avec des cha\xEEnes" ---- - -# Fonctions pour travailler avec des chaînes {#functions-for-working-with-strings} - -## vide {#empty} - -Renvoie 1 pour une chaîne vide ou 0 pour une chaîne non vide. -Le type de résultat est UInt8. -Une chaîne est considérée comme non vide si elle contient au moins un octet, même s'il s'agit d'un espace ou d'un octet nul. -La fonction fonctionne également pour les tableaux. - -## notEmpty {#notempty} - -Renvoie 0 pour une chaîne vide ou 1 pour une chaîne non vide. -Le type de résultat est UInt8. -La fonction fonctionne également pour les tableaux. - -## longueur {#length} - -Renvoie la longueur d'une chaîne en octets (pas en caractères, et pas en points de code). -Le type de résultat est UInt64. -La fonction fonctionne également pour les tableaux. - -## lengthUTF8 {#lengthutf8} - -Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). -Le type de résultat est UInt64. - -## char_length, CHAR_LENGTH {#char-length} - -Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). -Le type de résultat est UInt64. - -## character_length, CHARACTER_LENGTH {#character-length} - -Renvoie la longueur d'une chaîne en points de code Unicode (pas en caractères), en supposant que la chaîne contient un ensemble d'octets qui composent le texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). -Le type de résultat est UInt64. - -## plus bas, lcase {#lower} - -Convertit les symboles latins ASCII dans une chaîne en minuscules. - -## supérieur, ucase {#upper} - -Convertit les symboles latins ASCII dans une chaîne en majuscules. - -## lowerUTF8 {#lowerutf8} - -Convertit une chaîne en minuscules, en supposant que la chaîne de caractères contient un ensemble d'octets qui composent un texte UTF-8. -Il ne détecte pas la langue. Donc, pour le turc, le résultat pourrait ne pas être exactement correct. -Si la longueur de la séquence d'octets UTF-8 est différente pour les majuscules et les minuscules d'un point de code, le résultat peut être incorrect pour ce point de code. -Si la chaîne contient un ensemble d'octets qui N'est pas UTF-8, le comportement n'est pas défini. - -## upperUTF8 {#upperutf8} - -Convertit une chaîne en majuscules, en supposant que la chaîne de caractères contient un ensemble d'octets qui composent un texte UTF-8. -Il ne détecte pas la langue. Donc, pour le turc, le résultat pourrait ne pas être exactement correct. -Si la longueur de la séquence d'octets UTF-8 est différente pour les majuscules et les minuscules d'un point de code, le résultat peut être incorrect pour ce point de code. -Si la chaîne contient un ensemble d'octets qui N'est pas UTF-8, le comportement n'est pas défini. - -## isValidUTF8 {#isvalidutf8} - -Renvoie 1, si l'ensemble d'octets est codé en UTF-8 valide, sinon 0. - -## toValidUTF8 {#tovalidutf8} - -Remplace les caractères UTF-8 non valides par `�` (U+FFFD) caractère. Tous les caractères non valides s'exécutant dans une rangée sont réduits en un seul caractère de remplacement. - -``` sql -toValidUTF8( input_string ) -``` - -Paramètre: - -- input_string — Any set of bytes represented as the [Chaîne](../../sql-reference/data-types/string.md) type de données objet. - -Valeur renvoyée: chaîne UTF-8 valide. - -**Exemple** - -``` sql -SELECT toValidUTF8('\x61\xF0\x80\x80\x80b') -``` - -``` text -┌─toValidUTF8('a����b')─┐ -│ a�b │ -└───────────────────────┘ -``` - -## répéter {#repeat} - -Répète une corde autant de fois que spécifié et concatène les valeurs répliquées comme une seule chaîne. - -**Syntaxe** - -``` sql -repeat(s, n) -``` - -**Paramètre** - -- `s` — The string to repeat. [Chaîne](../../sql-reference/data-types/string.md). -- `n` — The number of times to repeat the string. [UInt](../../sql-reference/data-types/int-uint.md). - -**Valeur renvoyée** - -La chaîne unique, qui contient la chaîne `s` répéter `n` temps. Si `n` \< 1, la fonction renvoie une chaîne vide. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT repeat('abc', 10) -``` - -Résultat: - -``` text -┌─repeat('abc', 10)──────────────┐ -│ abcabcabcabcabcabcabcabcabcabc │ -└────────────────────────────────┘ -``` - -## inverser {#reverse} - -Inverse la chaîne (comme une séquence d'octets). - -## reverseUTF8 {#reverseutf8} - -Inverse une séquence de points de code Unicode, en supposant que la chaîne contient un ensemble d'octets représentant un texte UTF-8. Sinon, il fait autre chose (il ne lance pas d'exception). - -## format(pattern, s0, s1, …) {#format} - -Formatage du motif constant avec la chaîne listée dans les arguments. `pattern` est un modèle de format Python simplifié. Chaîne de Format contient “replacement fields” entouré par des accolades `{}`. Tout ce qui n'est pas contenu dans les accolades est considéré comme du texte littéral, qui est copié inchangé dans la sortie. Si vous devez inclure un caractère d'Accolade dans le texte littéral, il peut être échappé en doublant: `{{ '{{' }}` et `{{ '}}' }}`. Les noms de champs peuvent être des nombres (à partir de zéro) ou vides (ils sont alors traités comme des nombres de conséquence). - -``` sql -SELECT format('{1} {0} {1}', 'World', 'Hello') -``` - -``` text -┌─format('{1} {0} {1}', 'World', 'Hello')─┐ -│ Hello World Hello │ -└─────────────────────────────────────────┘ -``` - -``` sql -SELECT format('{} {}', 'Hello', 'World') -``` - -``` text -┌─format('{} {}', 'Hello', 'World')─┐ -│ Hello World │ -└───────────────────────────────────┘ -``` - -## concat {#concat} - -Concatène les chaînes répertoriées dans les arguments, sans séparateur. - -**Syntaxe** - -``` sql -concat(s1, s2, ...) -``` - -**Paramètre** - -Valeurs de type String ou FixedString. - -**Valeurs renvoyées** - -Renvoie la chaîne qui résulte de la concaténation des arguments. - -Si l'une des valeurs d'argument est `NULL`, `concat` retourner `NULL`. - -**Exemple** - -Requête: - -``` sql -SELECT concat('Hello, ', 'World!') -``` - -Résultat: - -``` text -┌─concat('Hello, ', 'World!')─┐ -│ Hello, World! │ -└─────────────────────────────┘ -``` - -## concatAssumeInjective {#concatassumeinjective} - -Même que [concat](#concat) la différence est que vous devez vous assurer que `concat(s1, s2, ...) → sn` est injectif, il sera utilisé pour l'optimisation du groupe par. - -La fonction est nommée “injective” si elle renvoie toujours un résultat différent pour différentes valeurs d'arguments. En d'autres termes: des arguments différents ne donnent jamais un résultat identique. - -**Syntaxe** - -``` sql -concatAssumeInjective(s1, s2, ...) -``` - -**Paramètre** - -Valeurs de type String ou FixedString. - -**Valeurs renvoyées** - -Renvoie la chaîne qui résulte de la concaténation des arguments. - -Si l'une des valeurs d'argument est `NULL`, `concatAssumeInjective` retourner `NULL`. - -**Exemple** - -Table d'entrée: - -``` sql -CREATE TABLE key_val(`key1` String, `key2` String, `value` UInt32) ENGINE = TinyLog; -INSERT INTO key_val VALUES ('Hello, ','World',1), ('Hello, ','World',2), ('Hello, ','World!',3), ('Hello',', World!',2); -SELECT * from key_val; -``` - -``` text -┌─key1────┬─key2─────┬─value─┐ -│ Hello, │ World │ 1 │ -│ Hello, │ World │ 2 │ -│ Hello, │ World! │ 3 │ -│ Hello │ , World! │ 2 │ -└─────────┴──────────┴───────┘ -``` - -Requête: - -``` sql -SELECT concat(key1, key2), sum(value) FROM key_val GROUP BY concatAssumeInjective(key1, key2) -``` - -Résultat: - -``` text -┌─concat(key1, key2)─┬─sum(value)─┐ -│ Hello, World! │ 3 │ -│ Hello, World! │ 2 │ -│ Hello, World │ 3 │ -└────────────────────┴────────────┘ -``` - -## substring(s, offset, longueur), mid(s, offset, longueur), substr(s, offset, longueur) {#substring} - -Renvoie une sous-chaîne commençant par l'octet du ‘offset’ index ‘length’ octets de long. L'indexation des caractères commence à partir d'un (comme dans SQL standard). Le ‘offset’ et ‘length’ les arguments doivent être des constantes. - -## substringUTF8(s, offset, longueur) {#substringutf8} - -Le même que ‘substring’, mais pour les points de code Unicode. Fonctionne sous l'hypothèse que la chaîne contient un ensemble d'octets représentant un texte codé en UTF-8. Si cette hypothèse n'est pas remplie, elle renvoie un résultat (elle ne lance pas d'exception). - -## appendTrailingCharIfAbsent (s, c) {#appendtrailingcharifabsent} - -Si l' ‘s’ la chaîne n'est pas vide et ne contient pas ‘c’ personnage à la fin, il ajoute le ‘c’ personnage à la fin. - -## convertCharset(s, à partir de, à) {#convertcharset} - -Retourne une chaîne de caractères ‘s’ qui a été converti à partir de l'encodage dans ‘from’ pour l'encodage dans ‘to’. - -## base64Encode(s) {#base64encode} - -Encodage ‘s’ chaîne dans base64 - -## base64Decode(s) {#base64decode} - -Décoder la chaîne codée en base64 ‘s’ dans la chaîne d'origine. En cas d'échec, une exception est levée. - -## tryBase64Decode(s) {#trybase64decode} - -Semblable à base64Decode, mais en cas d'erreur, une chaîne vide serait renvoyé. - -## endsWith (s, suffixe) {#endswith} - -Renvoie s'il faut se terminer par le suffixe spécifié. Retourne 1 si la chaîne se termine par le suffixe spécifié, sinon elle renvoie 0. - -## startsWith (STR, préfixe) {#startswith} - -Retourne 1 si la chaîne commence par le préfixe spécifié, sinon elle renvoie 0. - -``` sql -SELECT startsWith('Spider-Man', 'Spi'); -``` - -**Valeurs renvoyées** - -- 1, si la chaîne commence par le préfixe spécifié. -- 0, si la chaîne ne commence pas par le préfixe spécifié. - -**Exemple** - -Requête: - -``` sql -SELECT startsWith('Hello, world!', 'He'); -``` - -Résultat: - -``` text -┌─startsWith('Hello, world!', 'He')─┐ -│ 1 │ -└───────────────────────────────────┘ -``` - -## coupe {#trim} - -Supprime tous les caractères spécifiés du début ou de la fin d'une chaîne. -Par défaut supprime toutes les occurrences consécutives d'espaces communs (caractère ASCII 32) des deux extrémités d'une chaîne. - -**Syntaxe** - -``` sql -trim([[LEADING|TRAILING|BOTH] trim_character FROM] input_string) -``` - -**Paramètre** - -- `trim_character` — specified characters for trim. [Chaîne](../../sql-reference/data-types/string.md). -- `input_string` — string for trim. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée** - -Une chaîne sans caractères de début et (ou) de fin spécifiés. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT trim(BOTH ' ()' FROM '( Hello, world! )') -``` - -Résultat: - -``` text -┌─trim(BOTH ' ()' FROM '( Hello, world! )')─┐ -│ Hello, world! │ -└───────────────────────────────────────────────┘ -``` - -## trimLeft {#trimleft} - -Supprime toutes les occurrences consécutives d'espaces communs (caractère ASCII 32) depuis le début d'une chaîne. Il ne supprime pas d'autres types de caractères d'espaces (tabulation, espace sans pause, etc.). - -**Syntaxe** - -``` sql -trimLeft(input_string) -``` - -Alias: `ltrim(input_string)`. - -**Paramètre** - -- `input_string` — string to trim. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée** - -Une chaîne sans ouvrir les espaces communs. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT trimLeft(' Hello, world! ') -``` - -Résultat: - -``` text -┌─trimLeft(' Hello, world! ')─┐ -│ Hello, world! │ -└─────────────────────────────────────┘ -``` - -## trimRight {#trimright} - -Supprime toutes les occurrences consécutives d'espaces communs (caractère ASCII 32) de la fin d'une chaîne. Il ne supprime pas d'autres types de caractères d'espaces (tabulation, espace sans pause, etc.). - -**Syntaxe** - -``` sql -trimRight(input_string) -``` - -Alias: `rtrim(input_string)`. - -**Paramètre** - -- `input_string` — string to trim. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée** - -Une chaîne sans espaces communs de fin. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT trimRight(' Hello, world! ') -``` - -Résultat: - -``` text -┌─trimRight(' Hello, world! ')─┐ -│ Hello, world! │ -└──────────────────────────────────────┘ -``` - -## trimBoth {#trimboth} - -Supprime toutes les occurrences consécutives d'espaces communs (caractère ASCII 32) des deux extrémités d'une chaîne. Il ne supprime pas d'autres types de caractères d'espaces (tabulation, espace sans pause, etc.). - -**Syntaxe** - -``` sql -trimBoth(input_string) -``` - -Alias: `trim(input_string)`. - -**Paramètre** - -- `input_string` — string to trim. [Chaîne](../../sql-reference/data-types/string.md). - -**Valeur renvoyée** - -Une chaîne sans espaces communs de début et de fin. - -Type: `String`. - -**Exemple** - -Requête: - -``` sql -SELECT trimBoth(' Hello, world! ') -``` - -Résultat: - -``` text -┌─trimBoth(' Hello, world! ')─┐ -│ Hello, world! │ -└─────────────────────────────────────┘ -``` - -## CRC32 (s) {#crc32} - -Renvoie la somme de contrôle CRC32 d'une chaîne, en utilisant le polynôme CRC-32-IEEE 802.3 et la valeur initiale `0xffffffff` (zlib mise en œuvre). - -Le type de résultat est UInt32. - -## CRC32IEEE (s) {#crc32ieee} - -Renvoie la somme de contrôle CRC32 d'une chaîne, en utilisant le polynôme CRC-32-IEEE 802.3. - -Le type de résultat est UInt32. - -## CRC64 (s) {#crc64} - -Renvoie la somme de contrôle CRC64 d'une chaîne, en utilisant le polynôme CRC-64-ECMA. - -Le type de résultat est UInt64. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/string_functions/) diff --git a/docs/fr/sql-reference/functions/string-replace-functions.md b/docs/fr/sql-reference/functions/string-replace-functions.md deleted file mode 100644 index 5389a2bc927..00000000000 --- a/docs/fr/sql-reference/functions/string-replace-functions.md +++ /dev/null @@ -1,94 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 42 -toc_title: "Pour remplacer dans les cha\xEEnes" ---- - -# Fonctions de recherche et de remplacement dans les chaînes {#functions-for-searching-and-replacing-in-strings} - -## replaceOne(botte de foin, modèle, remplacement) {#replaceonehaystack-pattern-replacement} - -Remplace la première occurrence, si elle existe, ‘pattern’ sous-chaîne dans ‘haystack’ avec l' ‘replacement’ substring. -Ci-après, ‘pattern’ et ‘replacement’ doivent être constantes. - -## replaceAll(botte de foin, motif, remplacement), Remplacer(botte de foin, motif, remplacement) {#replaceallhaystack-pattern-replacement-replacehaystack-pattern-replacement} - -Remplace toutes les occurrences du ‘pattern’ sous-chaîne dans ‘haystack’ avec l' ‘replacement’ substring. - -## replaceRegexpOne(botte de foin, modèle, remplacement) {#replaceregexponehaystack-pattern-replacement} - -Remplacement en utilisant le ‘pattern’ expression régulière. Une expression régulière re2. -Remplace seulement la première occurrence, si elle existe. -Un motif peut être spécifié comme ‘replacement’. Ce modèle peut inclure des substitutions `\0-\9`. -Substitution `\0` inclut l'expression régulière entière. Substitution `\1-\9` correspond au sous-modèle numbers.To utilisez le `\` caractère dans un modèle, échappez-le en utilisant `\`. -Aussi garder à l'esprit qu'un littéral de chaîne nécessite une évasion. - -Exemple 1. Conversion de la date au format américain: - -``` sql -SELECT DISTINCT - EventDate, - replaceRegexpOne(toString(EventDate), '(\\d{4})-(\\d{2})-(\\d{2})', '\\2/\\3/\\1') AS res -FROM test.hits -LIMIT 7 -FORMAT TabSeparated -``` - -``` text -2014-03-17 03/17/2014 -2014-03-18 03/18/2014 -2014-03-19 03/19/2014 -2014-03-20 03/20/2014 -2014-03-21 03/21/2014 -2014-03-22 03/22/2014 -2014-03-23 03/23/2014 -``` - -Exemple 2. Copier une chaîne dix fois: - -``` sql -SELECT replaceRegexpOne('Hello, World!', '.*', '\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0') AS res -``` - -``` text -┌─res────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World!Hello, World! │ -└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## replaceRegexpAll(botte de foin, modèle, remplacement) {#replaceregexpallhaystack-pattern-replacement} - -Cela fait la même chose, mais remplace toutes les occurrences. Exemple: - -``` sql -SELECT replaceRegexpAll('Hello, World!', '.', '\\0\\0') AS res -``` - -``` text -┌─res────────────────────────┐ -│ HHeelllloo,, WWoorrlldd!! │ -└────────────────────────────┘ -``` - -Par exception, si une expression régulière travaillé sur un vide sous-chaîne, le remplacement n'est pas effectué plus d'une fois. -Exemple: - -``` sql -SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res -``` - -``` text -┌─res─────────────────┐ -│ here: Hello, World! │ -└─────────────────────┘ -``` - -## regexpQuoteMeta (s) {#regexpquotemetas} - -La fonction ajoute une barre oblique inverse avant certains caractères prédéfinis dans la chaîne. -Les personnages prédéfinis: ‘0’, ‘\\’, ‘\|’, ‘(’, ‘)’, ‘^’, ‘$’, ‘.’, ‘\[’, '\]', ‘?’, '\*‘,’+‘,’{‘,’:‘,’-'. -Cette implémentation diffère légèrement de re2:: RE2:: QuoteMeta. Il échappe à zéro octet comme \\0 au lieu de 00 et il échappe uniquement les caractères requis. -Pour plus d'informations, voir le lien: [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473) - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/string_replace_functions/) diff --git a/docs/fr/sql-reference/functions/string-search-functions.md b/docs/fr/sql-reference/functions/string-search-functions.md deleted file mode 100644 index 20217edd32c..00000000000 --- a/docs/fr/sql-reference/functions/string-search-functions.md +++ /dev/null @@ -1,379 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: "Pour Rechercher Des Cha\xEEnes" ---- - -# Fonctions de recherche de chaînes {#functions-for-searching-strings} - -La recherche est sensible à la casse par défaut dans toutes ces fonctions. Il existe des variantes pour la recherche insensible à la casse. - -## position(botte de foin, aiguille), localiser( botte de foin, aiguille) {#position} - -Renvoie la position (en octets) de la sous-chaîne trouvée dans la chaîne, à partir de 1. - -Fonctionne sous l'hypothèse que la chaîne de caractères contient un ensemble d'octets représentant un octet texte codé. Si cette hypothèse n'est pas remplie et qu'un caractère ne peut pas être représenté à l'aide d'un seul octet, la fonction ne lance pas d'exception et renvoie un résultat inattendu. Si le caractère peut être représenté en utilisant deux octets, il utilisera deux octets et ainsi de suite. - -Pour une recherche insensible à la casse, utilisez la fonction [positioncaseinsensible](#positioncaseinsensitive). - -**Syntaxe** - -``` sql -position(haystack, needle) -``` - -Alias: `locate(haystack, needle)`. - -**Paramètre** - -- `haystack` — string, in which substring will to be searched. [Chaîne](../syntax.md#syntax-string-literal). -- `needle` — substring to be searched. [Chaîne](../syntax.md#syntax-string-literal). - -**Valeurs renvoyées** - -- Position de départ en octets (à partir de 1), si la sous-chaîne a été trouvée. -- 0, si la sous-chaîne n'a pas été trouvé. - -Type: `Integer`. - -**Exemple** - -Phrase “Hello, world!” contient un ensemble d'octets représentant un octet texte codé. La fonction renvoie un résultat attendu: - -Requête: - -``` sql -SELECT position('Hello, world!', '!') -``` - -Résultat: - -``` text -┌─position('Hello, world!', '!')─┐ -│ 13 │ -└────────────────────────────────┘ -``` - -La même phrase en russe contient des caractères qui ne peuvent pas être représentés en utilisant un seul octet. La fonction renvoie un résultat inattendu (utilisation [positionUTF8](#positionutf8) fonction pour le texte codé sur plusieurs octets): - -Requête: - -``` sql -SELECT position('Привет, мир!', '!') -``` - -Résultat: - -``` text -┌─position('Привет, мир!', '!')─┐ -│ 21 │ -└───────────────────────────────┘ -``` - -## positioncaseinsensible {#positioncaseinsensitive} - -Le même que [position](#position) renvoie la position (en octets) de la sous-chaîne trouvée dans la chaîne, à partir de 1. Utilisez la fonction pour une recherche insensible à la casse. - -Fonctionne sous l'hypothèse que la chaîne de caractères contient un ensemble d'octets représentant un octet texte codé. Si cette hypothèse n'est pas remplie et qu'un caractère ne peut pas être représenté à l'aide d'un seul octet, la fonction ne lance pas d'exception et renvoie un résultat inattendu. Si le caractère peut être représenté en utilisant deux octets, il utilisera deux octets et ainsi de suite. - -**Syntaxe** - -``` sql -positionCaseInsensitive(haystack, needle) -``` - -**Paramètre** - -- `haystack` — string, in which substring will to be searched. [Chaîne](../syntax.md#syntax-string-literal). -- `needle` — substring to be searched. [Chaîne](../syntax.md#syntax-string-literal). - -**Valeurs renvoyées** - -- Position de départ en octets (à partir de 1), si la sous-chaîne a été trouvée. -- 0, si la sous-chaîne n'a pas été trouvé. - -Type: `Integer`. - -**Exemple** - -Requête: - -``` sql -SELECT positionCaseInsensitive('Hello, world!', 'hello') -``` - -Résultat: - -``` text -┌─positionCaseInsensitive('Hello, world!', 'hello')─┐ -│ 1 │ -└───────────────────────────────────────────────────┘ -``` - -## positionUTF8 {#positionutf8} - -Renvoie la position (en points Unicode) de la sous-chaîne trouvée dans la chaîne, à partir de 1. - -Fonctionne sous l'hypothèse que la chaîne contient un ensemble d'octets représentant un texte codé en UTF-8. Si cette hypothèse n'est pas remplie, la fonction ne lance pas d'exception et renvoie un résultat inattendu. Si le caractère peut être représenté en utilisant deux points Unicode, il en utilisera deux et ainsi de suite. - -Pour une recherche insensible à la casse, utilisez la fonction [positionCaseInsensitiveUTF8](#positioncaseinsensitiveutf8). - -**Syntaxe** - -``` sql -positionUTF8(haystack, needle) -``` - -**Paramètre** - -- `haystack` — string, in which substring will to be searched. [Chaîne](../syntax.md#syntax-string-literal). -- `needle` — substring to be searched. [Chaîne](../syntax.md#syntax-string-literal). - -**Valeurs renvoyées** - -- Position de départ dans les points Unicode (à partir de 1), si la sous-chaîne a été trouvée. -- 0, si la sous-chaîne n'a pas été trouvé. - -Type: `Integer`. - -**Exemple** - -Phrase “Hello, world!” en russe contient un ensemble de points Unicode représentant un texte codé à un seul point. La fonction renvoie un résultat attendu: - -Requête: - -``` sql -SELECT positionUTF8('Привет, мир!', '!') -``` - -Résultat: - -``` text -┌─positionUTF8('Привет, мир!', '!')─┐ -│ 12 │ -└───────────────────────────────────┘ -``` - -Phrase “Salut, étudiante!” où le caractère `é` peut être représenté en utilisant un point (`U+00E9`) ou deux points (`U+0065U+0301`) la fonction peut être retournée un résultat inattendu: - -Requête pour la lettre `é` qui est représenté un point Unicode `U+00E9`: - -``` sql -SELECT positionUTF8('Salut, étudiante!', '!') -``` - -Résultat: - -``` text -┌─positionUTF8('Salut, étudiante!', '!')─┐ -│ 17 │ -└────────────────────────────────────────┘ -``` - -Requête pour la lettre `é` qui est représenté deux points Unicode `U+0065U+0301`: - -``` sql -SELECT positionUTF8('Salut, étudiante!', '!') -``` - -Résultat: - -``` text -┌─positionUTF8('Salut, étudiante!', '!')─┐ -│ 18 │ -└────────────────────────────────────────┘ -``` - -## positionCaseInsensitiveUTF8 {#positioncaseinsensitiveutf8} - -Le même que [positionUTF8](#positionutf8) mais est sensible à la casse. Renvoie la position (en points Unicode) de la sous-chaîne trouvée dans la chaîne, à partir de 1. - -Fonctionne sous l'hypothèse que la chaîne contient un ensemble d'octets représentant un texte codé en UTF-8. Si cette hypothèse n'est pas remplie, la fonction ne lance pas d'exception et renvoie un résultat inattendu. Si le caractère peut être représenté en utilisant deux points Unicode, il en utilisera deux et ainsi de suite. - -**Syntaxe** - -``` sql -positionCaseInsensitiveUTF8(haystack, needle) -``` - -**Paramètre** - -- `haystack` — string, in which substring will to be searched. [Chaîne](../syntax.md#syntax-string-literal). -- `needle` — substring to be searched. [Chaîne](../syntax.md#syntax-string-literal). - -**Valeur renvoyée** - -- Position de départ dans les points Unicode (à partir de 1), si la sous-chaîne a été trouvée. -- 0, si la sous-chaîne n'a pas été trouvé. - -Type: `Integer`. - -**Exemple** - -Requête: - -``` sql -SELECT positionCaseInsensitiveUTF8('Привет, мир!', 'Мир') -``` - -Résultat: - -``` text -┌─positionCaseInsensitiveUTF8('Привет, мир!', 'Мир')─┐ -│ 9 │ -└────────────────────────────────────────────────────┘ -``` - -## multirecherchallpositions {#multisearchallpositions} - -Le même que [position](string-search-functions.md#position) mais les retours `Array` des positions (en octets) des sous-chaînes correspondantes trouvées dans la chaîne. Les Positions sont indexées à partir de 1. - -La recherche est effectuée sur des séquences d'octets sans tenir compte de l'encodage et du classement des chaînes. - -- Pour une recherche ASCII insensible à la casse, utilisez la fonction `multiSearchAllPositionsCaseInsensitive`. -- Pour la recherche en UTF-8, Utilisez la fonction [multiSearchAllPositionsUTF8](#multiSearchAllPositionsUTF8). -- Pour la recherche UTF-8 insensible à la casse, utilisez la fonction multiSearchAllPositionsCaseInsensitiveutf8. - -**Syntaxe** - -``` sql -multiSearchAllPositions(haystack, [needle1, needle2, ..., needlen]) -``` - -**Paramètre** - -- `haystack` — string, in which substring will to be searched. [Chaîne](../syntax.md#syntax-string-literal). -- `needle` — substring to be searched. [Chaîne](../syntax.md#syntax-string-literal). - -**Valeurs renvoyées** - -- Tableau de positions de départ en octets (à partir de 1), si la sous-chaîne correspondante a été trouvée et 0 si elle n'est pas trouvée. - -**Exemple** - -Requête: - -``` sql -SELECT multiSearchAllPositions('Hello, World!', ['hello', '!', 'world']) -``` - -Résultat: - -``` text -┌─multiSearchAllPositions('Hello, World!', ['hello', '!', 'world'])─┐ -│ [0,13,0] │ -└───────────────────────────────────────────────────────────────────┘ -``` - -## multiSearchAllPositionsUTF8 {#multiSearchAllPositionsUTF8} - -Voir `multiSearchAllPositions`. - -## multiSearchFirstPosition(botte de foin, \[aiguille1, aiguille2, …, needleet\]) {#multisearchfirstposition} - -Le même que `position` mais renvoie le décalage le plus à gauche de la chaîne `haystack` et qui correspond à certains des aiguilles. - -Pour une recherche insensible à la casse ou/et au format UTF-8, utilisez les fonctions `multiSearchFirstPositionCaseInsensitive, multiSearchFirstPositionUTF8, multiSearchFirstPositionCaseInsensitiveUTF8`. - -## multiSearchFirstIndex(botte de foin, \[aiguille1, aiguille2, …, needleet\]) {#multisearchfirstindexhaystack-needle1-needle2-needlen} - -Renvoie l'index `i` (à partir de 1) de l'aiguille trouvée la plus à gaucheje dans la chaîne `haystack` et 0 sinon. - -Pour une recherche insensible à la casse ou/et au format UTF-8, utilisez les fonctions `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`. - -## multiSearchAny(botte de foin, \[aiguille1, aiguille2, …, needleet\]) {#function-multisearchany} - -Renvoie 1, si au moins une aiguille de chaîneje correspond à la chaîne `haystack` et 0 sinon. - -Pour une recherche insensible à la casse ou/et au format UTF-8, utilisez les fonctions `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`. - -!!! note "Note" - Dans tous les `multiSearch*` fonctions le nombre d'aiguilles doit être d'au moins 28 en raison de la spécification de mise en œuvre. - -## match (botte de foin, motif) {#matchhaystack-pattern} - -Vérifie si la chaîne correspond au `pattern` expression régulière. Un `re2` expression régulière. Le [syntaxe](https://github.com/google/re2/wiki/Syntax) de la `re2` les expressions régulières sont plus limitées que la syntaxe des expressions régulières Perl. - -Renvoie 0 si elle ne correspond pas, ou 1 si elle correspond. - -Notez que le symbole antislash (`\`) est utilisé pour s'échapper dans l'expression régulière. Le même symbole est utilisé pour échapper dans les littéraux de chaîne. Donc, pour échapper au symbole dans une expression régulière, vous devez écrire deux barres obliques inverses ( \\ ) dans un littéral de chaîne. - -L'expression régulière travaille à la chaîne, comme si c'est un ensemble d'octets. L'expression régulière ne peut pas contenir d'octets nuls. -Pour que les modèles recherchent des sous-chaînes dans une chaîne, il est préférable D'utiliser LIKE ou ‘position’ depuis ils travaillent beaucoup plus vite. - -## multiMatchAny(botte de foin, \[motif1, modèle2, …, patternet\]) {#multimatchanyhaystack-pattern1-pattern2-patternn} - -Le même que `match` mais renvoie 0 si aucune des expressions régulières sont appariés et 1 si l'un des modèles les matchs. Il utilise [hyperscan](https://github.com/intel/hyperscan) bibliothèque. Pour que les modèles recherchent des sous-chaînes dans une chaîne, il est préférable d'utiliser `multiSearchAny` comme cela fonctionne beaucoup plus vite. - -!!! note "Note" - La longueur de l'un des `haystack` la chaîne doit être inférieure à 232 octets sinon l'exception est levée. Cette restriction a lieu en raison de l'API hyperscan. - -## multiMatchAnyIndex(botte de foin, \[motif1, modèle2, …, patternet\]) {#multimatchanyindexhaystack-pattern1-pattern2-patternn} - -Le même que `multiMatchAny` mais retourne un index qui correspond à la botte de foin. - -## multiMatchAllIndices(botte de foin, \[motif1, modèle2, …, patternet\]) {#multimatchallindiceshaystack-pattern1-pattern2-patternn} - -Le même que `multiMatchAny`, mais renvoie le tableau de tous les indices qui correspondent à la botte de foin dans n'importe quel ordre. - -## multiFuzzyMatchAny(botte de foin, distance, \[motif1, modèle2, …, patternet\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} - -Le même que `multiMatchAny`, mais renvoie 1 si un motif correspond à la botte de foin dans une constante [distance d'édition](https://en.wikipedia.org/wiki/Edit_distance). Cette fonction est également en mode expérimental et peut être extrêmement lente. Pour plus d'informations, voir [documentation hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching). - -## multiFuzzyMatchAnyIndex(botte de foin, distance, \[motif1, modèle2, …, patternet\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} - -Le même que `multiFuzzyMatchAny`, mais renvoie tout index qui correspond à la botte de foin à une distance d'édition constante. - -## multiFuzzyMatchAllIndices(botte de foin, distance, \[motif1, modèle2, …, patternet\]) {#multifuzzymatchallindiceshaystack-distance-pattern1-pattern2-patternn} - -Le même que `multiFuzzyMatchAny`, mais renvoie le tableau de tous les indices dans n'importe quel ordre qui correspond à la botte de foin à une distance d'édition constante. - -!!! note "Note" - `multiFuzzyMatch*` les fonctions ne prennent pas en charge les expressions régulières UTF-8, et ces expressions sont traitées comme des octets en raison de la restriction hyperscan. - -!!! note "Note" - Pour désactiver toutes les fonctions qui utilisent hyperscan, utilisez le réglage `SET allow_hyperscan = 0;`. - -## extrait(botte de foin, motif) {#extracthaystack-pattern} - -Extraits d'un fragment d'une chaîne à l'aide d'une expression régulière. Si ‘haystack’ ne correspond pas à l' ‘pattern’ regex, une chaîne vide est renvoyée. Si l'expression rationnelle ne contient pas de sous-modèles, elle prend le fragment qui correspond à l'expression rationnelle entière. Sinon, il prend le fragment qui correspond au premier sous-masque. - -## extractAll(botte de foin, motif) {#extractallhaystack-pattern} - -Extrait tous les fragments d'une chaîne à l'aide d'une expression régulière. Si ‘haystack’ ne correspond pas à l' ‘pattern’ regex, une chaîne vide est renvoyée. Renvoie un tableau de chaînes composé de toutes les correspondances à l'expression rationnelle. En général, le comportement est le même que le ‘extract’ fonction (il prend le premier sous-masque, ou l'expression entière s'il n'y a pas de sous-masque). - -## comme (botte de foin, motif), botte de foin comme opérateur de motif {#function-like} - -Vérifie si une chaîne correspond à une expression régulière simple. -L'expression régulière peut contenir les métasymboles `%` et `_`. - -`%` indique n'importe quelle quantité d'octets (y compris zéro caractère). - -`_` indique un octet. - -Utilisez la barre oblique inverse (`\`) pour échapper aux métasymboles. Voir la note sur l'échappement dans la description du ‘match’ fonction. - -Pour les expressions régulières comme `%needle%`, le code est plus optimale et fonctionne aussi vite que le `position` fonction. -Pour d'autres expressions régulières, le code est le même que pour la ‘match’ fonction. - -## notLike (botte de foin, motif), botte de foin pas comme opérateur de motif {#function-notlike} - -La même chose que ‘like’ mais négative. - -## ngramDistance(botte de foin, aiguille) {#ngramdistancehaystack-needle} - -Calcule la distance de 4 grammes entre `haystack` et `needle`: counts the symmetric difference between two multisets of 4-grams and normalizes it by the sum of their cardinalities. Returns float number from 0 to 1 – the closer to zero, the more strings are similar to each other. If the constant `needle` ou `haystack` est plus de 32Kb, jette une exception. Si une partie de la non-constante `haystack` ou `needle` les chaînes sont plus que 32Kb, la distance est toujours un. - -Pour une recherche insensible à la casse ou/et au format UTF-8, utilisez les fonctions `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`. - -## ngramSearch(botte de foin, aiguille) {#ngramsearchhaystack-needle} - -Même que `ngramDistance` mais calcule la différence non symétrique entre `needle` et `haystack` – the number of n-grams from needle minus the common number of n-grams normalized by the number of `needle` n-grammes. Le plus proche d'un, le plus probable `needle` est dans le `haystack`. Peut être utile pour la recherche de chaîne floue. - -Pour une recherche insensible à la casse ou/et au format UTF-8, utilisez les fonctions `ngramSearchCaseInsensitive, ngramSearchUTF8, ngramSearchCaseInsensitiveUTF8`. - -!!! note "Note" - For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables – collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function – we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one – this works for Latin and mostly for all Cyrillic letters. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) diff --git a/docs/fr/sql-reference/functions/type-conversion-functions.md b/docs/fr/sql-reference/functions/type-conversion-functions.md deleted file mode 100644 index c17b24c69dc..00000000000 --- a/docs/fr/sql-reference/functions/type-conversion-functions.md +++ /dev/null @@ -1,534 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: La Conversion De Type ---- - -# Fonctions De Conversion De Type {#type-conversion-functions} - -## Problèmes courants des Conversions numériques {#numeric-conversion-issues} - -Lorsque vous convertissez une valeur d'un type de données à un autre, vous devez vous rappeler que dans le cas courant, il s'agit d'une opération dangereuse qui peut entraîner une perte de données. Une perte de données peut se produire si vous essayez d'ajuster la valeur d'un type de données plus grand à un type de données plus petit, ou si vous convertissez des valeurs entre différents types de données. - -ClickHouse a le [même comportement que les programmes C++ ](https://en.cppreference.com/w/cpp/language/implicit_conversion). - -## toInt (8/16/32/64) {#toint8163264} - -Convertit une valeur d'entrée en [Int](../../sql-reference/data-types/int-uint.md) type de données. Cette fonction comprend: - -- `toInt8(expr)` — Results in the `Int8` type de données. -- `toInt16(expr)` — Results in the `Int16` type de données. -- `toInt32(expr)` — Results in the `Int32` type de données. -- `toInt64(expr)` — Results in the `Int64` type de données. - -**Paramètre** - -- `expr` — [Expression](../syntax.md#syntax-expressions) renvoyer un nombre ou une chaîne avec la représentation décimale d'un nombre. Les représentations binaires, octales et hexadécimales des nombres ne sont pas prises en charge. Les zéros principaux sont dépouillés. - -**Valeur renvoyée** - -Valeur entière dans le `Int8`, `Int16`, `Int32`, ou `Int64` type de données. - -Fonctions d'utilisation [l'arrondi vers zéro](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), ce qui signifie qu'ils tronquent des chiffres fractionnaires de nombres. - -Le comportement des fonctions pour le [NaN et Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments est indéfini. Rappelez-vous sur [problèmes de conversion numérique](#numeric-conversion-issues), lorsque vous utilisez les fonctions. - -**Exemple** - -``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) -``` - -``` text -┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ -│ -9223372036854775808 │ 32 │ 16 │ 8 │ -└──────────────────────┴─────────────┴───────────────┴─────────────┘ -``` - -## toInt (8/16/32/64)OrZero {#toint8163264orzero} - -Il prend un argument de type String et essaie de l'analyser en Int (8 \| 16 \| 32 \| 64). En cas d'échec, renvoie 0. - -**Exemple** - -``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123') -``` - -``` text -┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ -│ 123123 │ 0 │ -└─────────────────────────┴───────────────────────────┘ -``` - -## toInt (8/16/32/64)OrNull {#toint8163264ornull} - -Il prend un argument de type String et essaie de l'analyser en Int (8 \| 16 \| 32 \| 64). En cas d'échec, renvoie NULL. - -**Exemple** - -``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123') -``` - -``` text -┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ -│ 123123 │ ᴺᵁᴸᴸ │ -└─────────────────────────┴───────────────────────────┘ -``` - -## toUInt (8/16/32/64) {#touint8163264} - -Convertit une valeur d'entrée en [UInt](../../sql-reference/data-types/int-uint.md) type de données. Cette fonction comprend: - -- `toUInt8(expr)` — Results in the `UInt8` type de données. -- `toUInt16(expr)` — Results in the `UInt16` type de données. -- `toUInt32(expr)` — Results in the `UInt32` type de données. -- `toUInt64(expr)` — Results in the `UInt64` type de données. - -**Paramètre** - -- `expr` — [Expression](../syntax.md#syntax-expressions) renvoyer un nombre ou une chaîne avec la représentation décimale d'un nombre. Les représentations binaires, octales et hexadécimales des nombres ne sont pas prises en charge. Les zéros principaux sont dépouillés. - -**Valeur renvoyée** - -Valeur entière dans le `UInt8`, `UInt16`, `UInt32`, ou `UInt64` type de données. - -Fonctions d'utilisation [l'arrondi vers zéro](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), ce qui signifie qu'ils tronquent des chiffres fractionnaires de nombres. - -Le comportement des fonctions pour les agruments négatifs et pour le [NaN et Inf](../../sql-reference/data-types/float.md#data_type-float-nan-inf) arguments est indéfini. Si vous passez une chaîne avec un nombre négatif, par exemple `'-32'`, ClickHouse soulève une exception. Rappelez-vous sur [problèmes de conversion numérique](#numeric-conversion-issues), lorsque vous utilisez les fonctions. - -**Exemple** - -``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) -``` - -``` text -┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ -│ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ -└─────────────────────┴───────────────┴────────────────┴──────────────┘ -``` - -## toUInt (8/16/32/64)OrZero {#touint8163264orzero} - -## toUInt (8/16/32/64)OrNull {#touint8163264ornull} - -## toFloat (32/64) {#tofloat3264} - -## toFloat (32/64)OrZero {#tofloat3264orzero} - -## toFloat (32/64) OrNull {#tofloat3264ornull} - -## jour {#todate} - -## toDateOrZero {#todateorzero} - -## toDateOrNull {#todateornull} - -## toDateTime {#todatetime} - -## toDateTimeOrZero {#todatetimeorzero} - -## toDateTimeOrNull {#todatetimeornull} - -## toDecimal (32/64/128) {#todecimal3264128} - -Convertir `value` à l' [Décimal](../../sql-reference/data-types/decimal.md) type de données avec précision de `S`. Le `value` peut être un nombre ou une chaîne. Le `S` (l'échelle) paramètre spécifie le nombre de décimales. - -- `toDecimal32(value, S)` -- `toDecimal64(value, S)` -- `toDecimal128(value, S)` - -## toDecimal (32/64/128) OrNull {#todecimal3264128ornull} - -Convertit une chaîne d'entrée en [Nullable (Décimal (P, S))](../../sql-reference/data-types/decimal.md) valeur de type de données. Cette famille de fonctions comprennent: - -- `toDecimal32OrNull(expr, S)` — Results in `Nullable(Decimal32(S))` type de données. -- `toDecimal64OrNull(expr, S)` — Results in `Nullable(Decimal64(S))` type de données. -- `toDecimal128OrNull(expr, S)` — Results in `Nullable(Decimal128(S))` type de données. - -Ces fonctions devraient être utilisées à la place de `toDecimal*()` fonctions, si vous préférez obtenir un `NULL` la valeur au lieu d'une exception dans le cas d'une valeur d'entrée erreur d'analyse. - -**Paramètre** - -- `expr` — [Expression](../syntax.md#syntax-expressions), retourne une valeur dans l' [Chaîne](../../sql-reference/data-types/string.md) type de données. ClickHouse attend la représentation textuelle du nombre décimal. Exemple, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. - -**Valeur renvoyée** - -Une valeur dans l' `Nullable(Decimal(P,S))` type de données. La valeur contient: - -- Numéro `S` décimales, si ClickHouse interprète la chaîne d'entrée comme un nombre. -- `NULL` si ClickHouse ne peut pas interpréter la chaîne d'entrée comme un nombre ou si le nombre d'entrée contient plus de `S` décimale. - -**Exemple** - -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) -``` - -``` text -┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ -│ -1.11100 │ Nullable(Decimal(9, 5)) │ -└──────────┴────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) -``` - -``` text -┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ -│ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ -└──────┴────────────────────────────────────────────────────┘ -``` - -## toDecimal (32/64/128)OrZero {#todecimal3264128orzero} - -Convertit une valeur d'entrée en [Decimal(P,S)](../../sql-reference/data-types/decimal.md) type de données. Cette famille de fonctions comprennent: - -- `toDecimal32OrZero( expr, S)` — Results in `Decimal32(S)` type de données. -- `toDecimal64OrZero( expr, S)` — Results in `Decimal64(S)` type de données. -- `toDecimal128OrZero( expr, S)` — Results in `Decimal128(S)` type de données. - -Ces fonctions devraient être utilisées à la place de `toDecimal*()` fonctions, si vous préférez obtenir un `0` la valeur au lieu d'une exception dans le cas d'une valeur d'entrée erreur d'analyse. - -**Paramètre** - -- `expr` — [Expression](../syntax.md#syntax-expressions), retourne une valeur dans l' [Chaîne](../../sql-reference/data-types/string.md) type de données. ClickHouse attend la représentation textuelle du nombre décimal. Exemple, `'1.111'`. -- `S` — Scale, the number of decimal places in the resulting value. - -**Valeur renvoyée** - -Une valeur dans l' `Nullable(Decimal(P,S))` type de données. La valeur contient: - -- Numéro `S` décimales, si ClickHouse interprète la chaîne d'entrée comme un nombre. -- 0 avec `S` décimales, si ClickHouse ne peut pas interpréter la chaîne d'entrée comme un nombre ou si le nombre d'entrée contient plus de `S` décimale. - -**Exemple** - -``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val) -``` - -``` text -┌──────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ -│ -1.11100 │ Decimal(9, 5) │ -└──────────┴────────────────────────────────────────────────────┘ -``` - -``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val) -``` - -``` text -┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ -│ 0.00 │ Decimal(9, 2) │ -└──────┴────────────────────────────────────────────────────┘ -``` - -## toString {#tostring} - -Fonctions de conversion entre des nombres, des chaînes (mais pas des chaînes fixes), des dates et des dates avec des heures. -Toutes ces fonctions acceptent un argument. - -Lors de la conversion vers ou à partir d'une chaîne, la valeur est formatée ou analysée en utilisant les mêmes règles que pour le format TabSeparated (et presque tous les autres formats de texte). Si la chaîne ne peut pas être analysée, une exception est levée et la demande est annulée. - -Lors de la conversion de dates en nombres ou vice versa, la date correspond au nombre de jours depuis le début de L'époque Unix. -Lors de la conversion de dates avec des heures en nombres ou vice versa, la date avec l'heure correspond au nombre de secondes depuis le début de L'époque Unix. - -Les formats date et date-avec-heure pour les fonctions toDate/toDateTime sont définis comme suit: - -``` text -YYYY-MM-DD -YYYY-MM-DD hh:mm:ss -``` - -À titre d'exception, si vous convertissez des types numériques UInt32, Int32, UInt64 ou Int64 à Date, et si le nombre est supérieur ou égal à 65536, le nombre est interprété comme un horodatage Unix (et non comme le nombre de jours) et est arrondi à la date. Cela permet de prendre en charge l'occurrence commune de l'écriture ‘toDate(unix_timestamp)’, qui autrement serait une erreur et nécessiterait d'écrire le plus lourd ‘toDate(toDateTime(unix_timestamp))’. - -La Conversion entre une date et une date avec l'heure est effectuée de manière naturelle: en ajoutant une heure nulle ou en supprimant l'heure. - -La Conversion entre types numériques utilise les mêmes règles que les affectations entre différents types numériques en C++. - -De plus, la fonction ToString de L'argument DateTime peut prendre un deuxième argument de chaîne contenant le nom du fuseau horaire. Exemple: `Asia/Yekaterinburg` Dans ce cas, l'heure est formatée en fonction du fuseau horaire spécifié. - -``` sql -SELECT - now() AS now_local, - toString(now(), 'Asia/Yekaterinburg') AS now_yekat -``` - -``` text -┌───────────now_local─┬─now_yekat───────────┐ -│ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ -└─────────────────────┴─────────────────────┘ -``` - -Voir aussi l' `toUnixTimestamp` fonction. - -## toFixedString (s, N) {#tofixedstrings-n} - -Convertit un argument de type String en un type FixedString (N) (une chaîne de longueur fixe N). N doit être une constante. -Si la chaîne a moins d'octets que N, elle est complétée avec des octets null à droite. Si la chaîne a plus d'octets que N, une exception est levée. - -## toStringCutToZero(s) {#tostringcuttozeros} - -Accepte un argument String ou FixedString. Renvoie la chaîne avec le contenu tronqué au premier octet zéro trouvé. - -Exemple: - -``` sql -SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut -``` - -``` text -┌─s─────────────┬─s_cut─┐ -│ foo\0\0\0\0\0 │ foo │ -└───────────────┴───────┘ -``` - -``` sql -SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut -``` - -``` text -┌─s──────────┬─s_cut─┐ -│ foo\0bar\0 │ foo │ -└────────────┴───────┘ -``` - -## reinterpretAsUInt (8/16/32/64) {#reinterpretasuint8163264} - -## reinterpretAsInt (8/16/32/64) {#reinterpretasint8163264} - -## reinterpretAsFloat (32/64) {#reinterpretasfloat3264} - -## réinterprétasdate {#reinterpretasdate} - -## reinterpretAsDateTime {#reinterpretasdatetime} - -Ces fonctions acceptent une chaîne et interprètent les octets placés au début de la chaîne comme un nombre dans l'ordre de l'hôte (little endian). Si la chaîne n'est pas assez longue, les fonctions fonctionnent comme si la chaîne était remplie avec le nombre nécessaire d'octets nuls. Si la chaîne est plus longue que nécessaire, les octets supplémentaires sont ignorés. Une date est interprétée comme le nombre de jours depuis le début de l'Époque Unix, et une date avec le temps, est interprété comme le nombre de secondes écoulées depuis le début de l'Époque Unix. - -## reinterpretAsString {#type_conversion_functions-reinterpretAsString} - -Cette fonction accepte un nombre ou une date ou une date avec l'heure, et renvoie une chaîne contenant des octets représentant la valeur correspondante dans l'ordre de l'hôte (little endian). Les octets nuls sont supprimés de la fin. Par exemple, une valeur de type uint32 de 255 est une chaîne longue d'un octet. - -## reinterpretAsFixedString {#reinterpretasfixedstring} - -Cette fonction accepte un nombre ou une date ou une date avec l'heure, et renvoie une chaîne fixe contenant des octets représentant la valeur correspondante dans l'ordre de l'hôte (little endian). Les octets nuls sont supprimés de la fin. Par exemple, une valeur de type uint32 de 255 est une chaîne fixe longue d'un octet. - -## CAST (x, T) {#type_conversion_function-cast} - -Convertir ‘x’ à l' ‘t’ type de données. La syntaxe CAST (X comme t) est également prise en charge. - -Exemple: - -``` sql -SELECT - '2016-06-15 23:00:00' AS timestamp, - CAST(timestamp AS DateTime) AS datetime, - CAST(timestamp AS Date) AS date, - CAST(timestamp, 'String') AS string, - CAST(timestamp, 'FixedString(22)') AS fixed_string -``` - -``` text -┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ -│ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ -└─────────────────────┴─────────────────────┴────────────┴─────────────────────┴───────────────────────────┘ -``` - -La Conversion en FixedString (N) ne fonctionne que pour les arguments de type String ou FixedString (N). - -Type conversion en [Nullable](../../sql-reference/data-types/nullable.md) et le dos est pris en charge. Exemple: - -``` sql -SELECT toTypeName(x) FROM t_null -``` - -``` text -┌─toTypeName(x)─┐ -│ Int8 │ -│ Int8 │ -└───────────────┘ -``` - -``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null -``` - -``` text -┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ -│ Nullable(UInt16) │ -│ Nullable(UInt16) │ -└─────────────────────────────────────────┘ -``` - -## toInterval (année / trimestre / Mois / Semaine / Jour / Heure / Minute / Seconde) {#function-tointerval} - -Convertit un argument de type Number en [Intervalle](../../sql-reference/data-types/special-data-types/interval.md) type de données. - -**Syntaxe** - -``` sql -toIntervalSecond(number) -toIntervalMinute(number) -toIntervalHour(number) -toIntervalDay(number) -toIntervalWeek(number) -toIntervalMonth(number) -toIntervalQuarter(number) -toIntervalYear(number) -``` - -**Paramètre** - -- `number` — Duration of interval. Positive integer number. - -**Valeurs renvoyées** - -- La valeur de `Interval` type de données. - -**Exemple** - -``` sql -WITH - toDate('2019-01-01') AS date, - INTERVAL 1 WEEK AS interval_week, - toIntervalWeek(1) AS interval_to_week -SELECT - date + interval_week, - date + interval_to_week -``` - -``` text -┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ -│ 2019-01-08 │ 2019-01-08 │ -└───────────────────────────┴──────────────────────────────┘ -``` - -## parseDateTimeBestEffort {#parsedatetimebesteffort} - -Convertit une date et une heure dans le [Chaîne](../../sql-reference/data-types/string.md) la représentation de [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) type de données. - -La fonction d'analyse [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 1123 - 5.2.14 RFC-822 date et heure Spécification](https://tools.ietf.org/html/rfc1123#page-55), ClickHouse et d'autres formats de date et d'heure. - -**Syntaxe** - -``` sql -parseDateTimeBestEffort(time_string [, time_zone]); -``` - -**Paramètre** - -- `time_string` — String containing a date and time to convert. [Chaîne](../../sql-reference/data-types/string.md). -- `time_zone` — Time zone. The function parses `time_string` selon le fuseau horaire. [Chaîne](../../sql-reference/data-types/string.md). - -**Formats non standard pris en charge** - -- Une chaîne contenant 9..10 chiffres [le timestamp unix](https://en.wikipedia.org/wiki/Unix_time). -- Une chaîne avec une date et une heure composant: `YYYYMMDDhhmmss`, `DD/MM/YYYY hh:mm:ss`, `DD-MM-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. -- Une chaîne avec une date, mais pas de composant de temps: `YYYY`, `YYYYMM`, `YYYY*MM`, `DD/MM/YYYY`, `DD-MM-YY` etc. -- Une chaîne avec un jour et une heure: `DD`, `DD hh`, `DD hh:mm`. Dans ce cas `YYYY-MM` sont substitués comme suit `2000-01`. -- Une chaîne qui inclut la date et l'heure ainsi que des informations de décalage de fuseau horaire: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. Exemple, `2020-12-12 17:36:00 -5:00`. - -Pour tous les formats avec séparateur, la fonction analyse les noms de mois exprimés par leur nom complet ou par les trois premières lettres d'un nom de mois. Exemple: `24/DEC/18`, `24-Dec-18`, `01-September-2018`. - -**Valeur renvoyée** - -- `time_string` converti à l' `DateTime` type de données. - -**Exemple** - -Requête: - -``` sql -SELECT parseDateTimeBestEffort('12/12/2020 12:12:57') -AS parseDateTimeBestEffort; -``` - -Résultat: - -``` text -┌─parseDateTimeBestEffort─┐ -│ 2020-12-12 12:12:57 │ -└─────────────────────────┘ -``` - -Requête: - -``` sql -SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort -``` - -Résultat: - -``` text -┌─parseDateTimeBestEffort─┐ -│ 2018-08-18 10:22:16 │ -└─────────────────────────┘ -``` - -Requête: - -``` sql -SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort -``` - -Résultat: - -``` text -┌─parseDateTimeBestEffort─┐ -│ 2015-07-07 12:04:41 │ -└─────────────────────────┘ -``` - -Requête: - -``` sql -SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort -``` - -Résultat: - -``` text -┌─parseDateTimeBestEffort─┐ -│ 2018-12-12 10:12:12 │ -└─────────────────────────┘ -``` - -Requête: - -``` sql -SELECT parseDateTimeBestEffort('10 20:19') -``` - -Résultat: - -``` text -┌─parseDateTimeBestEffort('10 20:19')─┐ -│ 2000-01-10 20:19:00 │ -└─────────────────────────────────────┘ -``` - -**Voir Aussi** - -- \[Annonce ISO 8601 par @xkcd\](https://xkcd.com/1179/) -- [RFC 1123](https://tools.ietf.org/html/rfc1123) -- [jour](#todate) -- [toDateTime](#todatetime) - -## parseDateTimeBestEffortOrNull {#parsedatetimebesteffortornull} - -De même que pour [parseDateTimeBestEffort](#parsedatetimebesteffort) sauf qu'il renvoie null lorsqu'il rencontre un format de date qui ne peut pas être traité. - -## parseDateTimeBestEffortOrZero {#parsedatetimebesteffortorzero} - -De même que pour [parseDateTimeBestEffort](#parsedatetimebesteffort) sauf qu'il renvoie une date zéro ou une date zéro lorsqu'il rencontre un format de date qui ne peut pas être traité. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/type_conversion_functions/) diff --git a/docs/fr/sql-reference/functions/url-functions.md b/docs/fr/sql-reference/functions/url-functions.md deleted file mode 100644 index 2bb2203a10b..00000000000 --- a/docs/fr/sql-reference/functions/url-functions.md +++ /dev/null @@ -1,209 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 54 -toc_title: Travailler avec des URL ---- - -# Fonctions pour travailler avec des URL {#functions-for-working-with-urls} - -Toutes ces fonctions ne suivent pas la RFC. Ils sont simplifiés au maximum pour améliorer les performances. - -## Fonctions qui extraient des parties d'une URL {#functions-that-extract-parts-of-a-url} - -Si la partie pertinente n'est pas présente dans une URL, une chaîne vide est renvoyée. - -### protocole {#protocol} - -Extrait le protocole d'une URL. - -Examples of typical returned values: http, https, ftp, mailto, tel, magnet… - -### domaine {#domain} - -Extrait le nom d'hôte d'une URL. - -``` sql -domain(url) -``` - -**Paramètre** - -- `url` — URL. Type: [Chaîne](../../sql-reference/data-types/string.md). - -L'URL peut être spécifiée avec ou sans schéma. Exemple: - -``` text -svn+ssh://some.svn-hosting.com:80/repo/trunk -some.svn-hosting.com:80/repo/trunk -https://yandex.com/time/ -``` - -Pour ces exemples, le `domain` la fonction renvoie les résultats suivants: - -``` text -some.svn-hosting.com -some.svn-hosting.com -yandex.com -``` - -**Valeurs renvoyées** - -- Nom d'hôte. Si ClickHouse peut analyser la chaîne d'entrée en tant QU'URL. -- Chaîne vide. Si ClickHouse ne peut pas analyser la chaîne d'entrée en tant QU'URL. - -Type: `String`. - -**Exemple** - -``` sql -SELECT domain('svn+ssh://some.svn-hosting.com:80/repo/trunk') -``` - -``` text -┌─domain('svn+ssh://some.svn-hosting.com:80/repo/trunk')─┐ -│ some.svn-hosting.com │ -└────────────────────────────────────────────────────────┘ -``` - -### domainWithoutWWW {#domainwithoutwww} - -Renvoie le domaine et ne supprime pas plus d'un ‘www.’ dès le début de celui-ci, si présent. - -### topLevelDomain {#topleveldomain} - -Extrait le domaine de premier niveau d'une URL. - -``` sql -topLevelDomain(url) -``` - -**Paramètre** - -- `url` — URL. Type: [Chaîne](../../sql-reference/data-types/string.md). - -L'URL peut être spécifiée avec ou sans schéma. Exemple: - -``` text -svn+ssh://some.svn-hosting.com:80/repo/trunk -some.svn-hosting.com:80/repo/trunk -https://yandex.com/time/ -``` - -**Valeurs renvoyées** - -- Nom de domaine. Si ClickHouse peut analyser la chaîne d'entrée en tant QU'URL. -- Chaîne vide. Si ClickHouse ne peut pas analyser la chaîne d'entrée en tant QU'URL. - -Type: `String`. - -**Exemple** - -``` sql -SELECT topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk') -``` - -``` text -┌─topLevelDomain('svn+ssh://www.some.svn-hosting.com:80/repo/trunk')─┐ -│ com │ -└────────────────────────────────────────────────────────────────────┘ -``` - -### firstSignificantSubdomain {#firstsignificantsubdomain} - -Renvoie la “first significant subdomain”. C'est un concept non standard spécifique à Yandex.Metrica. Le premier sous-domaine significatif est un domaine de deuxième niveau s'il est ‘com’, ‘net’, ‘org’, ou ‘co’. Sinon, il est un domaine de troisième niveau. Exemple, `firstSignificantSubdomain (‘https://news.yandex.ru/’) = ‘yandex’, firstSignificantSubdomain (‘https://news.yandex.com.tr/’) = ‘yandex’`. La liste des “insignificant” les domaines de deuxième niveau et d'autres détails de mise en œuvre peuvent changer à l'avenir. - -### cutToFirstSignificantSubdomain {#cuttofirstsignificantsubdomain} - -Renvoie la partie du domaine qui inclut les sous-domaines de premier niveau “first significant subdomain” (voir l'explication ci-dessus). - -Exemple, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`. - -### chemin {#path} - -Retourne le chemin d'accès. Exemple: `/top/news.html` Le chemin n'inclut pas la chaîne de requête. - -### pathFull {#pathfull} - -La même chose que ci-dessus, mais y compris la chaîne de requête et le fragment. Exemple: / top / nouvelles.le html?page = 2 # commentaires - -### queryString {#querystring} - -Retourne la chaîne de requête. Exemple: page = 1 & lr=213. query-string n'inclut pas le point d'interrogation initial, ainsi que # et tout ce qui suit #. - -### fragment {#fragment} - -Renvoie l'identificateur de fragment. fragment n'inclut pas le symbole de hachage initial. - -### queryStringAndFragment {#querystringandfragment} - -Renvoie la chaîne de requête et l'Identificateur de fragment. Exemple: page = 1 # 29390. - -### extractURLParameter (URL, nom) {#extracturlparameterurl-name} - -Renvoie la valeur de la ‘name’ paramètre dans l'URL, le cas échéant. Sinon, une chaîne vide. S'il y a beaucoup de paramètres avec ce nom, il renvoie la première occurrence. Cette fonction fonctionne en supposant que le nom du paramètre est codé dans L'URL exactement de la même manière que dans l'argument passé. - -### extractURLParameters (URL) {#extracturlparametersurl} - -Renvoie un tableau de chaînes name = value correspondant aux paramètres D'URL. Les valeurs ne sont en aucun cas décodées. - -### extractURLParameterNames (URL) {#extracturlparameternamesurl} - -Retourne un tableau de chaînes de noms correspondant aux noms des paramètres d'URL. Les valeurs ne sont en aucun cas décodées. - -### URLHierarchy (URL) {#urlhierarchyurl} - -Retourne un tableau contenant L'URL, tronquée à la fin par les symboles /,? dans le chemin et la chaîne de requête. Les caractères séparateurs consécutifs sont comptés comme un. La coupe est faite dans la position après tous les caractères de séparation consécutifs. - -### URLPathHierarchy (URL) {#urlpathhierarchyurl} - -La même chose que ci-dessus, mais sans le protocole et l'hôte dans le résultat. Le / les élément (racine) n'est pas inclus. Exemple: la fonction est utilisée pour implémenter l'arborescence des rapports de L'URL dans Yandex. Métrique. - -``` text -URLPathHierarchy('https://example.com/browse/CONV-6788') = -[ - '/browse/', - '/browse/CONV-6788' -] -``` - -### decodeURLComponent (URL) {#decodeurlcomponenturl} - -Renvoie L'URL décodée. -Exemple: - -``` sql -SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS DecodedURL; -``` - -``` text -┌─DecodedURL─────────────────────────────┐ -│ http://127.0.0.1:8123/?query=SELECT 1; │ -└────────────────────────────────────────┘ -``` - -## Fonctions qui suppriment une partie D'une URL {#functions-that-remove-part-of-a-url} - -Si L'URL n'a rien de similaire, L'URL reste inchangée. - -### cutWWW {#cutwww} - -Supprime pas plus d'une ‘www.’ depuis le début du domaine de L'URL, s'il est présent. - -### cutQueryString {#cutquerystring} - -Supprime la chaîne de requête. Le point d'interrogation est également supprimé. - -### cutFragment {#cutfragment} - -Supprime l'identificateur de fragment. Le signe est également supprimé. - -### couperystringandfragment {#cutquerystringandfragment} - -Supprime la chaîne de requête et l'Identificateur de fragment. Le point d'interrogation et le signe numérique sont également supprimés. - -### cutURLParameter (URL, nom) {#cuturlparameterurl-name} - -Supprime le ‘name’ Paramètre URL, si présent. Cette fonction fonctionne en supposant que le nom du paramètre est codé dans L'URL exactement de la même manière que dans l'argument passé. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/url_functions/) diff --git a/docs/fr/sql-reference/functions/uuid-functions.md b/docs/fr/sql-reference/functions/uuid-functions.md deleted file mode 100644 index 9f9eb67d3e9..00000000000 --- a/docs/fr/sql-reference/functions/uuid-functions.md +++ /dev/null @@ -1,122 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 53 -toc_title: Travailler avec UUID ---- - -# Fonctions pour travailler avec UUID {#functions-for-working-with-uuid} - -Les fonctions pour travailler avec UUID sont listées ci-dessous. - -## generateUUIDv4 {#uuid-function-generate} - -Génère le [UUID](../../sql-reference/data-types/uuid.md) de [la version 4](https://tools.ietf.org/html/rfc4122#section-4.4). - -``` sql -generateUUIDv4() -``` - -**Valeur renvoyée** - -La valeur de type UUID. - -**Exemple d'utilisation** - -Cet exemple montre la création d'une table avec la colonne de type UUID et l'insertion d'une valeur dans la table. - -``` sql -CREATE TABLE t_uuid (x UUID) ENGINE=TinyLog - -INSERT INTO t_uuid SELECT generateUUIDv4() - -SELECT * FROM t_uuid -``` - -``` text -┌────────────────────────────────────x─┐ -│ f4bf890f-f9dc-4332-ad5c-0c18e73f28e9 │ -└──────────────────────────────────────┘ -``` - -## toUUID (x) {#touuid-x} - -Convertit la valeur de type de chaîne en type UUID. - -``` sql -toUUID(String) -``` - -**Valeur renvoyée** - -La valeur de type UUID. - -**Exemple d'utilisation** - -``` sql -SELECT toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') AS uuid -``` - -``` text -┌─────────────────────────────────uuid─┐ -│ 61f0c404-5cb3-11e7-907b-a6006ad3dba0 │ -└──────────────────────────────────────┘ -``` - -## UUIDStringToNum {#uuidstringtonum} - -Accepte une chaîne contenant 36 caractères dans le format `xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx`, et le renvoie comme un ensemble d'octets dans un [FixedString (16)](../../sql-reference/data-types/fixedstring.md). - -``` sql -UUIDStringToNum(String) -``` - -**Valeur renvoyée** - -FixedString (16) - -**Exemples d'utilisation** - -``` sql -SELECT - '612f3c40-5d3b-217e-707b-6a546a3d7b29' AS uuid, - UUIDStringToNum(uuid) AS bytes -``` - -``` text -┌─uuid─────────────────────────────────┬─bytes────────────┐ -│ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ -└──────────────────────────────────────┴──────────────────┘ -``` - -## UUIDNumToString {#uuidnumtostring} - -Accepte un [FixedString (16)](../../sql-reference/data-types/fixedstring.md) valeur, et renvoie une chaîne contenant 36 caractères au format texte. - -``` sql -UUIDNumToString(FixedString(16)) -``` - -**Valeur renvoyée** - -Chaîne. - -**Exemple d'utilisation** - -``` sql -SELECT - 'a/<@];!~p{jTj={)' AS bytes, - UUIDNumToString(toFixedString(bytes, 16)) AS uuid -``` - -``` text -┌─bytes────────────┬─uuid─────────────────────────────────┐ -│ a/<@];!~p{jTj={) │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ -└──────────────────┴──────────────────────────────────────┘ -``` - -## Voir Aussi {#see-also} - -- [dictGetUUID](ext-dict-functions.md#ext_dict_functions-other) - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/uuid_function/) diff --git a/docs/fr/sql-reference/functions/ym-dict-functions.md b/docs/fr/sql-reference/functions/ym-dict-functions.md deleted file mode 100644 index f1e4461e24a..00000000000 --- a/docs/fr/sql-reference/functions/ym-dict-functions.md +++ /dev/null @@ -1,155 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 59 -toc_title: Travailler avec Yandex.Dictionnaires Metrica ---- - -# Fonctions pour travailler avec Yandex.Dictionnaires Metrica {#functions-for-working-with-yandex-metrica-dictionaries} - -Pour que les fonctions ci-dessous fonctionnent, la configuration du serveur doit spécifier les chemins et les adresses pour obtenir tous les Yandex.Dictionnaires Metrica. Les dictionnaires sont chargés au premier appel de l'une de ces fonctions. Si les listes de référence ne peuvent pas être chargées, une exception est levée. - -Pour plus d'informations sur la création de listes de références, consultez la section “Dictionaries”. - -## Plusieurs Geobases {#multiple-geobases} - -ClickHouse soutient le travail avec plusieurs géobases alternatives (hiérarchies régionales) simultanément, afin de soutenir diverses perspectives sur les pays auxquels appartiennent certaines régions. - -Le ‘clickhouse-server’ config spécifie le fichier avec l'échelon régional::`/opt/geo/regions_hierarchy.txt` - -Outre ce fichier, il recherche également les fichiers à proximité qui ont le symbole _ et tout suffixe ajouté au nom (avant l'extension de fichier). -Par exemple, il trouvera également le fichier `/opt/geo/regions_hierarchy_ua.txt` si présente. - -`ua` est appelée la clé du dictionnaire. Pour un dictionnaire sans suffixe, la clé est une chaîne vide. - -Tous les dictionnaires sont rechargés dans l'exécution (une fois toutes les secondes, comme défini dans le paramètre de configuration builtin_dictionaries_reload_interval, ou une fois par heure par défaut). Cependant, la liste des dictionnaires disponibles est définie une fois, lorsque le serveur démarre. - -All functions for working with regions have an optional argument at the end – the dictionary key. It is referred to as the geobase. -Exemple: - -``` sql -regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txt -regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txt -regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt -``` - -### regionToCity (id \[, geobase\]) {#regiontocityid-geobase} - -Accepts a UInt32 number – the region ID from the Yandex geobase. If this region is a city or part of a city, it returns the region ID for the appropriate city. Otherwise, returns 0. - -### regionToArea (id \[, geobase\]) {#regiontoareaid-geobase} - -Convertit une région en une zone (tapez 5 dans la géobase). Dans tous les autres cas, cette fonction est la même que ‘regionToCity’. - -``` sql -SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) -FROM system.numbers -LIMIT 15 -``` - -``` text -┌─regionToName(regionToArea(toUInt32(number), \'ua\'))─┐ -│ │ -│ Moscow and Moscow region │ -│ St. Petersburg and Leningrad region │ -│ Belgorod region │ -│ Ivanovsk region │ -│ Kaluga region │ -│ Kostroma region │ -│ Kursk region │ -│ Lipetsk region │ -│ Orlov region │ -│ Ryazan region │ -│ Smolensk region │ -│ Tambov region │ -│ Tver region │ -│ Tula region │ -└──────────────────────────────────────────────────────┘ -``` - -### regionToDistrict(id \[, geobase\]) {#regiontodistrictid-geobase} - -Convertit une région en district fédéral (type 4 dans la géobase). Dans tous les autres cas, cette fonction est la même que ‘regionToCity’. - -``` sql -SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua')) -FROM system.numbers -LIMIT 15 -``` - -``` text -┌─regionToName(regionToDistrict(toUInt32(number), \'ua\'))─┐ -│ │ -│ Central federal district │ -│ Northwest federal district │ -│ South federal district │ -│ North Caucases federal district │ -│ Privolga federal district │ -│ Ural federal district │ -│ Siberian federal district │ -│ Far East federal district │ -│ Scotland │ -│ Faroe Islands │ -│ Flemish region │ -│ Brussels capital region │ -│ Wallonia │ -│ Federation of Bosnia and Herzegovina │ -└──────────────────────────────────────────────────────────┘ -``` - -### regionToCountry (id \[, geobase\]) {#regiontocountryid-geobase} - -Convertit une région en un pays. Dans tous les autres cas, cette fonction est la même que ‘regionToCity’. -Exemple: `regionToCountry(toUInt32(213)) = 225` convertit Moscou (213) en Russie (225). - -### regionToContinent(id \[, géobase\]) {#regiontocontinentid-geobase} - -Convertit une région en continent. Dans tous les autres cas, cette fonction est la même que ‘regionToCity’. -Exemple: `regionToContinent(toUInt32(213)) = 10001` convertit Moscou (213) en Eurasie (10001). - -### regionToTopContinent (#regiontotopcontinent) {#regiontotopcontinent-regiontotopcontinent} - -Trouve le continent le plus élevé dans la hiérarchie de la région. - -**Syntaxe** - -``` sql -regionToTopContinent(id[, geobase]); -``` - -**Paramètre** - -- `id` — Region ID from the Yandex geobase. [UInt32](../../sql-reference/data-types/int-uint.md). -- `geobase` — Dictionary key. See [Plusieurs Geobases](#multiple-geobases). [Chaîne](../../sql-reference/data-types/string.md). Facultatif. - -**Valeur renvoyée** - -- Identifiant du continent de haut niveau (ce dernier lorsque vous grimpez dans la hiérarchie des régions). -- 0, si il n'y a aucun. - -Type: `UInt32`. - -### regionToPopulation (id \[, geobase\]) {#regiontopopulationid-geobase} - -Obtient la population d'une région. -La population peut être enregistrée dans des fichiers avec la géobase. Voir la section “External dictionaries”. -Si la population n'est pas enregistrée pour la région, elle renvoie 0. -Dans la géobase Yandex, la population peut être enregistrée pour les régions enfants, mais pas pour les régions parentes. - -### regionIn(lhs, rhs \[, géobase\]) {#regioninlhs-rhs-geobase} - -Vérifie si un ‘lhs’ région appartient à une ‘rhs’ région. Renvoie un nombre UInt8 égal à 1 s'il appartient, Ou 0 s'il n'appartient pas. -The relationship is reflexive – any region also belongs to itself. - -### regionHierarchy (id \[, geobase\]) {#regionhierarchyid-geobase} - -Accepts a UInt32 number – the region ID from the Yandex geobase. Returns an array of region IDs consisting of the passed region and all parents along the chain. -Exemple: `regionHierarchy(toUInt32(213)) = [213,1,3,225,10001,10000]`. - -### regionToName(id \[, lang\]) {#regiontonameid-lang} - -Accepts a UInt32 number – the region ID from the Yandex geobase. A string with the name of the language can be passed as a second argument. Supported languages are: ru, en, ua, uk, by, kz, tr. If the second argument is omitted, the language ‘ru’ is used. If the language is not supported, an exception is thrown. Returns a string – the name of the region in the corresponding language. If the region with the specified ID doesn't exist, an empty string is returned. - -`ua` et `uk` les deux signifient ukrainien. - -[Article Original](https://clickhouse.tech/docs/en/query_language/functions/ym_dict_functions/) diff --git a/docs/fr/sql-reference/index.md b/docs/fr/sql-reference/index.md deleted file mode 100644 index 04e44892c05..00000000000 --- a/docs/fr/sql-reference/index.md +++ /dev/null @@ -1,20 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "R\xE9f\xE9rence SQL" -toc_hidden: true -toc_priority: 28 -toc_title: "cach\xE9s" ---- - -# Référence SQL {#sql-reference} - -ClickHouse prend en charge les types de requêtes suivants: - -- [SELECT](statements/select/index.md) -- [INSERT INTO](statements/insert-into.md) -- [CREATE](statements/create.md) -- [ALTER](statements/alter.md#query_language_queries_alter) -- [Autres types de requêtes](statements/misc.md) - -[Article Original](https://clickhouse.tech/docs/en/sql-reference/) diff --git a/docs/fr/sql-reference/operators/in.md b/docs/fr/sql-reference/operators/in.md deleted file mode 100644 index d87fe41a04f..00000000000 --- a/docs/fr/sql-reference/operators/in.md +++ /dev/null @@ -1,204 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -### Dans les opérateurs {#select-in-operators} - -Le `IN`, `NOT IN`, `GLOBAL IN`, et `GLOBAL NOT IN` les opérateurs sont traitées séparément, car leur fonctionnalité est assez riche. - -Le côté gauche de l'opérateur, soit une seule colonne ou un tuple. - -Exemple: - -``` sql -SELECT UserID IN (123, 456) FROM ... -SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... -``` - -Si le côté gauche est une colonne unique qui est dans l'index, et le côté droit est un ensemble de constantes, le système utilise l'index pour le traitement de la requête. - -Don't list too many values explicitly (i.e. millions). If a data set is large, put it in a temporary table (for example, see the section “External data for query processing”), puis utiliser une sous-requête. - -Le côté droit de l'opérateur peut être un ensemble d'expressions constantes, un ensemble de tuples avec des expressions constantes (illustrées dans les exemples ci-dessus), ou le nom d'une table de base de données ou une sous-requête SELECT entre parenthèses. - -Si le côté droit de l'opérateur est le nom d'une table (par exemple, `UserID IN users`), ceci est équivalent à la sous-requête `UserID IN (SELECT * FROM users)`. Utilisez ceci lorsque vous travaillez avec des données externes envoyées avec la requête. Par exemple, la requête peut être envoyée avec un ensemble d'ID utilisateur chargés dans le ‘users’ table temporaire, qui doit être filtrée. - -Si le côté droit de l'opérateur est un nom de table qui a le moteur Set (un ensemble de données préparé qui est toujours en RAM), l'ensemble de données ne sera pas créé à nouveau pour chaque requête. - -La sous-requête peut spécifier plusieurs colonnes pour filtrer les tuples. -Exemple: - -``` sql -SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... -``` - -Les colonnes à gauche et à droite de l'opérateur doit avoir le même type. - -L'opérateur IN et la sous-requête peuvent se produire dans n'importe quelle partie de la requête, y compris dans les fonctions d'agrégation et les fonctions lambda. -Exemple: - -``` sql -SELECT - EventDate, - avg(UserID IN - ( - SELECT UserID - FROM test.hits - WHERE EventDate = toDate('2014-03-17') - )) AS ratio -FROM test.hits -GROUP BY EventDate -ORDER BY EventDate ASC -``` - -``` text -┌──EventDate─┬────ratio─┐ -│ 2014-03-17 │ 1 │ -│ 2014-03-18 │ 0.807696 │ -│ 2014-03-19 │ 0.755406 │ -│ 2014-03-20 │ 0.723218 │ -│ 2014-03-21 │ 0.697021 │ -│ 2014-03-22 │ 0.647851 │ -│ 2014-03-23 │ 0.648416 │ -└────────────┴──────────┘ -``` - -Pour chaque jour après le 17 mars, comptez le pourcentage de pages vues par les utilisateurs qui ont visité le site le 17 mars. -Une sous-requête dans la clause est toujours exécuter une seule fois sur un seul serveur. Il n'y a pas de sous-requêtes dépendantes. - -## Le Traitement NULL {#null-processing-1} - -Pendant le traitement de la demande, l'opérateur n'assume que le résultat d'une opération avec [NULL](../syntax.md#null-literal) est toujours égale à `0` indépendamment de savoir si `NULL` est sur le côté droit ou gauche de l'opérateur. `NULL` les valeurs ne sont incluses dans aucun jeu de données, ne correspondent pas entre elles et ne peuvent pas être comparées. - -Voici un exemple avec le `t_null` table: - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 3 │ -└───┴──────┘ -``` - -L'exécution de la requête `SELECT x FROM t_null WHERE y IN (NULL,3)` vous donne le résultat suivant: - -``` text -┌─x─┐ -│ 2 │ -└───┘ -``` - -Vous pouvez voir que la ligne dans laquelle `y = NULL` est jeté hors de résultats de la requête. C'est parce que ClickHouse ne peut pas décider si `NULL` est inclus dans le `(NULL,3)` ensemble, les retours `0` comme le résultat de l'opération, et `SELECT` exclut cette ligne de la sortie finale. - -``` sql -SELECT y IN (NULL, 3) -FROM t_null -``` - -``` text -┌─in(y, tuple(NULL, 3))─┐ -│ 0 │ -│ 1 │ -└───────────────────────┘ -``` - -## Sous-Requêtes Distribuées {#select-distributed-subqueries} - -Il y a deux options pour IN-S avec des sous-requêtes (similaires aux jointures): normal `IN` / `JOIN` et `GLOBAL IN` / `GLOBAL JOIN`. Ils diffèrent dans la façon dont ils sont exécutés pour le traitement des requêtes distribuées. - -!!! attention "Attention" - Rappelez-vous que les algorithmes décrits ci-dessous peuvent travailler différemment en fonction de la [paramètre](../../operations/settings/settings.md) `distributed_product_mode` paramètre. - -Lors de l'utilisation de l'IN régulier, la requête est envoyée à des serveurs distants, et chacun d'eux exécute les sous-requêtes dans le `IN` ou `JOIN` clause. - -Lors de l'utilisation de `GLOBAL IN` / `GLOBAL JOINs`, d'abord toutes les sous-requêtes sont exécutées pour `GLOBAL IN` / `GLOBAL JOINs`, et les résultats sont recueillis dans des tableaux temporaires. Ensuite, les tables temporaires sont envoyés à chaque serveur distant, où les requêtes sont exécutées à l'aide temporaire de données. - -Pour une requête non distribuée, utilisez `IN` / `JOIN`. - -Soyez prudent lorsque vous utilisez des sous-requêtes dans le `IN` / `JOIN` clauses pour le traitement des requêtes distribuées. - -Regardons quelques exemples. Supposons que chaque serveur du cluster a un **local_table**. Chaque serveur dispose également d'une **table distributed_table** table avec le **Distribué** type, qui regarde tous les serveurs du cluster. - -Pour une requête à l' **table distributed_table**, la requête sera envoyée à tous les serveurs distants et exécutée sur eux en utilisant le **local_table**. - -Par exemple, la requête - -``` sql -SELECT uniq(UserID) FROM distributed_table -``` - -sera envoyé à tous les serveurs distants - -``` sql -SELECT uniq(UserID) FROM local_table -``` - -et l'exécuter sur chacun d'eux en parallèle, jusqu'à ce qu'il atteigne le stade où les résultats intermédiaires peuvent être combinés. Ensuite, les résultats intermédiaires seront retournés au demandeur de serveur et de fusion, et le résultat final sera envoyé au client. - -Examinons maintenant une requête avec IN: - -``` sql -SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) -``` - -- Calcul de l'intersection des audiences de deux sites. - -Cette requête sera envoyée à tous les serveurs distants - -``` sql -SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM local_table WHERE CounterID = 34) -``` - -En d'autres termes, l'ensemble de données de la clause IN sera collecté sur chaque serveur indépendamment, uniquement à travers les données stockées localement sur chacun des serveurs. - -Cela fonctionnera correctement et de manière optimale si vous êtes prêt pour ce cas et que vous avez réparti les données entre les serveurs de cluster de telle sorte que les données d'un seul ID utilisateur résident entièrement sur un seul serveur. Dans ce cas, toutes les données nécessaires seront disponibles localement sur chaque serveur. Sinon, le résultat sera erroné. Nous nous référons à cette variation de la requête que “local IN”. - -Pour corriger le fonctionnement de la requête lorsque les données sont réparties aléatoirement sur les serveurs de cluster, vous pouvez spécifier **table distributed_table** à l'intérieur d'une sous-requête. La requête ressemblerait à ceci: - -``` sql -SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) -``` - -Cette requête sera envoyée à tous les serveurs distants - -``` sql -SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) -``` - -La sous-requête commencera à s'exécuter sur chaque serveur distant. Étant donné que la sous-requête utilise une table distribuée, la sous-requête qui se trouve sur chaque serveur distant sera renvoyée à chaque serveur distant comme - -``` sql -SELECT UserID FROM local_table WHERE CounterID = 34 -``` - -Par exemple, si vous avez un cluster de 100 SERVEURS, l'exécution de la requête entière nécessitera 10 000 requêtes élémentaires, ce qui est généralement considéré comme inacceptable. - -Dans de tels cas, vous devez toujours utiliser GLOBAL IN au lieu de IN. Voyons comment cela fonctionne pour la requête - -``` sql -SELECT uniq(UserID) FROM distributed_table WHERE CounterID = 101500 AND UserID GLOBAL IN (SELECT UserID FROM distributed_table WHERE CounterID = 34) -``` - -Le serveur demandeur exécutera la sous requête - -``` sql -SELECT UserID FROM distributed_table WHERE CounterID = 34 -``` - -et le résultat sera mis dans une table temporaire en RAM. Ensuite, la demande sera envoyée à chaque serveur distant - -``` sql -SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 -``` - -et la table temporaire `_data1` sera envoyé à chaque serveur distant avec la requête (le nom de la table temporaire est défini par l'implémentation). - -Ceci est plus optimal que d'utiliser la normale dans. Cependant, gardez les points suivants à l'esprit: - -1. Lors de la création d'une table temporaire, les données ne sont pas uniques. Pour réduire le volume de données transmises sur le réseau, spécifiez DISTINCT dans la sous-requête. (Vous n'avez pas besoin de le faire pour un IN normal.) -2. La table temporaire sera envoyé à tous les serveurs distants. La Transmission ne tient pas compte de la topologie du réseau. Par exemple, si 10 serveurs distants résident dans un centre de données très distant par rapport au serveur demandeur, les données seront envoyées 10 fois sur le canal au centre de données distant. Essayez d'éviter les grands ensembles de données lorsque vous utilisez GLOBAL IN. -3. Lors de la transmission de données à des serveurs distants, les restrictions sur la bande passante réseau ne sont pas configurables. Vous pourriez surcharger le réseau. -4. Essayez de distribuer les données entre les serveurs afin que vous n'ayez pas besoin D'utiliser GLOBAL IN sur une base régulière. -5. Si vous devez utiliser GLOBAL in souvent, planifiez l'emplacement du cluster ClickHouse de sorte qu'un seul groupe de répliques ne réside pas dans plus d'un centre de données avec un réseau rapide entre eux, de sorte qu'une requête puisse être traitée entièrement dans un seul centre de données. - -Il est également judicieux de spécifier une table locale dans le `GLOBAL IN` clause, dans le cas où cette table locale est uniquement disponible sur le serveur demandeur et que vous souhaitez utiliser les données de celui-ci sur des serveurs distants. diff --git a/docs/fr/sql-reference/operators/index.md b/docs/fr/sql-reference/operators/index.md deleted file mode 100644 index 1635c7eece3..00000000000 --- a/docs/fr/sql-reference/operators/index.md +++ /dev/null @@ -1,277 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: "Op\xE9rateur" ---- - -# Opérateur {#operators} - -ClickHouse transforme les opérateurs en leurs fonctions correspondantes à l'étape d'analyse des requêtes en fonction de leur priorité, de leur priorité et de leur associativité. - -## Des Opérateurs D'Accès {#access-operators} - -`a[N]` – Access to an element of an array. The `arrayElement(a, N)` fonction. - -`a.N` – Access to a tuple element. The `tupleElement(a, N)` fonction. - -## Opérateur De Négation Numérique {#numeric-negation-operator} - -`-a` – The `negate (a)` fonction. - -## Opérateurs de Multiplication et de Division {#multiplication-and-division-operators} - -`a * b` – The `multiply (a, b)` fonction. - -`a / b` – The `divide(a, b)` fonction. - -`a % b` – The `modulo(a, b)` fonction. - -## Opérateurs d'Addition et de soustraction {#addition-and-subtraction-operators} - -`a + b` – The `plus(a, b)` fonction. - -`a - b` – The `minus(a, b)` fonction. - -## Opérateurs De Comparaison {#comparison-operators} - -`a = b` – The `equals(a, b)` fonction. - -`a == b` – The `equals(a, b)` fonction. - -`a != b` – The `notEquals(a, b)` fonction. - -`a <> b` – The `notEquals(a, b)` fonction. - -`a <= b` – The `lessOrEquals(a, b)` fonction. - -`a >= b` – The `greaterOrEquals(a, b)` fonction. - -`a < b` – The `less(a, b)` fonction. - -`a > b` – The `greater(a, b)` fonction. - -`a LIKE s` – The `like(a, b)` fonction. - -`a NOT LIKE s` – The `notLike(a, b)` fonction. - -`a BETWEEN b AND c` – The same as `a >= b AND a <= c`. - -`a NOT BETWEEN b AND c` – The same as `a < b OR a > c`. - -## Opérateurs pour travailler avec des ensembles de données {#operators-for-working-with-data-sets} - -*Voir [Dans les opérateurs](in.md).* - -`a IN ...` – The `in(a, b)` fonction. - -`a NOT IN ...` – The `notIn(a, b)` fonction. - -`a GLOBAL IN ...` – The `globalIn(a, b)` fonction. - -`a GLOBAL NOT IN ...` – The `globalNotIn(a, b)` fonction. - -## Opérateurs pour travailler avec des Dates et des heures {#operators-datetime} - -### EXTRACT {#operator-extract} - -``` sql -EXTRACT(part FROM date); -``` - -Extraire des parties d'une date donnée. Par exemple, vous pouvez récupérer un mois à partir d'une date donnée, ou d'une seconde à partir d'un moment. - -Le `part` paramètre spécifie la partie de la date à récupérer. Les valeurs suivantes sont disponibles: - -- `DAY` — The day of the month. Possible values: 1–31. -- `MONTH` — The number of a month. Possible values: 1–12. -- `YEAR` — The year. -- `SECOND` — The second. Possible values: 0–59. -- `MINUTE` — The minute. Possible values: 0–59. -- `HOUR` — The hour. Possible values: 0–23. - -Le `part` le paramètre est insensible à la casse. - -Le `date` paramètre spécifie la date ou l'heure à traiter. Soit [Date](../../sql-reference/data-types/date.md) ou [DateTime](../../sql-reference/data-types/datetime.md) le type est pris en charge. - -Exemple: - -``` sql -SELECT EXTRACT(DAY FROM toDate('2017-06-15')); -SELECT EXTRACT(MONTH FROM toDate('2017-06-15')); -SELECT EXTRACT(YEAR FROM toDate('2017-06-15')); -``` - -Dans l'exemple suivant, nous créons un tableau et de les insérer dans une valeur avec le `DateTime` type. - -``` sql -CREATE TABLE test.Orders -( - OrderId UInt64, - OrderName String, - OrderDate DateTime -) -ENGINE = Log; -``` - -``` sql -INSERT INTO test.Orders VALUES (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44')); -``` - -``` sql -SELECT - toYear(OrderDate) AS OrderYear, - toMonth(OrderDate) AS OrderMonth, - toDayOfMonth(OrderDate) AS OrderDay, - toHour(OrderDate) AS OrderHour, - toMinute(OrderDate) AS OrderMinute, - toSecond(OrderDate) AS OrderSecond -FROM test.Orders; -``` - -``` text -┌─OrderYear─┬─OrderMonth─┬─OrderDay─┬─OrderHour─┬─OrderMinute─┬─OrderSecond─┐ -│ 2008 │ 10 │ 11 │ 13 │ 23 │ 44 │ -└───────────┴────────────┴──────────┴───────────┴─────────────┴─────────────┘ -``` - -Vous pouvez voir plus d'exemples de [test](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00619_extract.sql). - -### INTERVAL {#operator-interval} - -Crée un [Intervalle](../../sql-reference/data-types/special-data-types/interval.md)- valeur de type qui doit être utilisée dans les opérations arithmétiques avec [Date](../../sql-reference/data-types/date.md) et [DateTime](../../sql-reference/data-types/datetime.md)-type de valeurs. - -Types d'intervalles: -- `SECOND` -- `MINUTE` -- `HOUR` -- `DAY` -- `WEEK` -- `MONTH` -- `QUARTER` -- `YEAR` - -!!! warning "Avertissement" - Les intervalles avec différents types ne peuvent pas être combinés. Vous ne pouvez pas utiliser des expressions comme `INTERVAL 4 DAY 1 HOUR`. Spécifiez des intervalles en unités inférieures ou égales à la plus petite unité de l'intervalle, par exemple, `INTERVAL 25 HOUR`. Vous pouvez utiliser les opérations consécutives, comme dans l'exemple ci-dessous. - -Exemple: - -``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR -``` - -``` text -┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ -└─────────────────────┴────────────────────────────────────────────────────────┘ -``` - -**Voir Aussi** - -- [Intervalle](../../sql-reference/data-types/special-data-types/interval.md) type de données -- [toInterval](../../sql-reference/functions/type-conversion-functions.md#function-tointerval) type fonctions de conversion - -## Opérateur De Négation Logique {#logical-negation-operator} - -`NOT a` – The `not(a)` fonction. - -## Logique ET de l'Opérateur {#logical-and-operator} - -`a AND b` – The`and(a, b)` fonction. - -## Logique ou opérateur {#logical-or-operator} - -`a OR b` – The `or(a, b)` fonction. - -## Opérateur Conditionnel {#conditional-operator} - -`a ? b : c` – The `if(a, b, c)` fonction. - -Note: - -L'opérateur conditionnel calcule les valeurs de b et c, puis vérifie si la condition a est remplie, puis renvoie la valeur correspondante. Si `b` ou `C` est un [arrayJoin()](../../sql-reference/functions/array-join.md#functions_arrayjoin) fonction, chaque ligne sera répliquée indépendamment de la “a” condition. - -## Expression Conditionnelle {#operator_case} - -``` sql -CASE [x] - WHEN a THEN b - [WHEN ... THEN ...] - [ELSE c] -END -``` - -Si `x` est spécifié, alors `transform(x, [a, ...], [b, ...], c)` function is used. Otherwise – `multiIf(a, b, ..., c)`. - -Si il n'y a pas de `ELSE c` dans l'expression, la valeur par défaut est `NULL`. - -Le `transform` la fonction ne fonctionne pas avec `NULL`. - -## Opérateur De Concaténation {#concatenation-operator} - -`s1 || s2` – The `concat(s1, s2) function.` - -## Opérateur De Création Lambda {#lambda-creation-operator} - -`x -> expr` – The `lambda(x, expr) function.` - -Les opérateurs suivants n'ont pas de priorité puisqu'ils sont des parenthèses: - -## Opérateur De Création De Tableau {#array-creation-operator} - -`[x1, ...]` – The `array(x1, ...) function.` - -## Opérateur De Création De Tuple {#tuple-creation-operator} - -`(x1, x2, ...)` – The `tuple(x2, x2, ...) function.` - -## Associativité {#associativity} - -Tous les opérateurs binaires ont associativité gauche. Exemple, `1 + 2 + 3` est transformé à `plus(plus(1, 2), 3)`. -Parfois, cela ne fonctionne pas de la façon que vous attendez. Exemple, `SELECT 4 > 2 > 3` résultat sera 0. - -Pour l'efficacité, le `and` et `or` les fonctions acceptent n'importe quel nombre d'arguments. Les chaînes de `AND` et `OR` les opérateurs se sont transformés en un seul appel de ces fonctions. - -## La vérification de `NULL` {#checking-for-null} - -Clickhouse soutient le `IS NULL` et `IS NOT NULL` opérateur. - -### IS NULL {#operator-is-null} - -- Pour [Nullable](../../sql-reference/data-types/nullable.md) type de valeurs, l' `IS NULL` opérateur retourne: - - `1` si la valeur est `NULL`. - - `0` autrement. -- Pour les autres valeurs, la `IS NULL` l'opérateur renvoie toujours `0`. - - - -``` sql -SELECT x+100 FROM t_null WHERE y IS NULL -``` - -``` text -┌─plus(x, 100)─┐ -│ 101 │ -└──────────────┘ -``` - -### IS NOT NULL {#is-not-null} - -- Pour [Nullable](../../sql-reference/data-types/nullable.md) type de valeurs, l' `IS NOT NULL` opérateur retourne: - - `0` si la valeur est `NULL`. - - `1` autrement. -- Pour les autres valeurs, la `IS NOT NULL` l'opérateur renvoie toujours `1`. - - - -``` sql -SELECT * FROM t_null WHERE y IS NOT NULL -``` - -``` text -┌─x─┬─y─┐ -│ 2 │ 3 │ -└───┴───┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/operators/) diff --git a/docs/fr/sql-reference/statements/alter.md b/docs/fr/sql-reference/statements/alter.md deleted file mode 100644 index 64fe21046a3..00000000000 --- a/docs/fr/sql-reference/statements/alter.md +++ /dev/null @@ -1,602 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 36 -toc_title: ALTER ---- - -## ALTER {#query_language_queries_alter} - -Le `ALTER` la requête est prise en charge uniquement pour `*MergeTree` des tables, ainsi que `Merge`et`Distributed`. La requête a plusieurs variantes. - -### Manipulations De Colonne {#column-manipulations} - -Modification de la structure de la table. - -``` sql -ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ... -``` - -Dans la requête, spécifiez une liste d'une ou plusieurs actions séparées par des virgules. -Chaque action est une opération sur une colonne. - -Les actions suivantes sont prises en charge: - -- [ADD COLUMN](#alter_add-column) — Adds a new column to the table. -- [DROP COLUMN](#alter_drop-column) — Deletes the column. -- [CLEAR COLUMN](#alter_clear-column) — Resets column values. -- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column. -- [MODIFY COLUMN](#alter_modify-column) — Changes column's type, default expression and TTL. - -Ces actions sont décrites en détail ci-dessous. - -#### ADD COLUMN {#alter_add-column} - -``` sql -ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after] -``` - -Ajoute une nouvelle colonne à la table spécifiée `name`, `type`, [`codec`](create.md#codecs) et `default_expr` (voir la section [Expressions par défaut](create.md#create-default-values)). - -Si l' `IF NOT EXISTS` la clause est incluse, la requête ne retournera pas d'erreur si la colonne existe déjà. Si vous spécifiez `AFTER name_after` (le nom d'une autre colonne), la colonne est ajoutée après celle spécifiée dans la liste des colonnes de la table. Sinon, la colonne est ajoutée à la fin de la table. Notez qu'il n'existe aucun moyen d'ajouter une colonne au début d'un tableau. Pour une chaîne d'actions, `name_after` peut être le nom d'une colonne est ajoutée dans l'une des actions précédentes. - -L'ajout d'une colonne modifie simplement la structure de la table, sans effectuer d'actions avec des données. Les données n'apparaissent pas sur le disque après la `ALTER`. Si les données sont manquantes pour une colonne lors de la lecture de la table, elles sont remplies avec des valeurs par défaut (en exécutant l'expression par défaut s'il y en a une, ou en utilisant des zéros ou des chaînes vides). La colonne apparaît sur le disque après la fusion des parties de données (voir [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)). - -Cette approche nous permet de compléter le `ALTER` requête instantanément, sans augmenter le volume de données anciennes. - -Exemple: - -``` sql -ALTER TABLE visits ADD COLUMN browser String AFTER user_id -``` - -#### DROP COLUMN {#alter_drop-column} - -``` sql -DROP COLUMN [IF EXISTS] name -``` - -Supprime la colonne avec le nom `name`. Si l' `IF EXISTS` la clause est spécifiée, la requête ne retournera pas d'erreur si la colonne n'existe pas. - -Supprime les données du système de fichiers. Comme cela supprime des fichiers entiers, la requête est terminée presque instantanément. - -Exemple: - -``` sql -ALTER TABLE visits DROP COLUMN browser -``` - -#### CLEAR COLUMN {#alter_clear-column} - -``` sql -CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name -``` - -Réinitialise toutes les données dans une colonne pour une partition spécifiée. En savoir plus sur la définition du nom de la partition dans la section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -Si l' `IF EXISTS` la clause est spécifiée, la requête ne retournera pas d'erreur si la colonne n'existe pas. - -Exemple: - -``` sql -ALTER TABLE visits CLEAR COLUMN browser IN PARTITION tuple() -``` - -#### COMMENT COLUMN {#alter_comment-column} - -``` sql -COMMENT COLUMN [IF EXISTS] name 'comment' -``` - -Ajoute un commentaire à la colonne. Si l' `IF EXISTS` la clause est spécifiée, la requête ne retournera pas d'erreur si la colonne n'existe pas. - -Chaque colonne peut avoir un commentaire. Si un commentaire existe déjà pour la colonne, un nouveau commentaire remplace le précédent commentaire. - -Les commentaires sont stockés dans le `comment_expression` colonne renvoyée par le [DESCRIBE TABLE](misc.md#misc-describe-table) requête. - -Exemple: - -``` sql -ALTER TABLE visits COMMENT COLUMN browser 'The table shows the browser used for accessing the site.' -``` - -#### MODIFY COLUMN {#alter_modify-column} - -``` sql -MODIFY COLUMN [IF EXISTS] name [type] [default_expr] [TTL] -``` - -Cette requête modifie le `name` les propriétés de la colonne: - -- Type - -- Expression par défaut - -- TTL - - For examples of columns TTL modifying, see [Column TTL](../engines/table_engines/mergetree_family/mergetree.md#mergetree-column-ttl). - -Si l' `IF EXISTS` la clause est spécifiée, la requête ne retournera pas d'erreur si la colonne n'existe pas. - -Lors de la modification du type, les valeurs sont converties comme si [toType](../../sql-reference/functions/type-conversion-functions.md) les fonctions ont été appliquées. Si seule l'expression par défaut est modifiée, la requête ne fait rien de complexe et est terminée presque instantanément. - -Exemple: - -``` sql -ALTER TABLE visits MODIFY COLUMN browser Array(String) -``` - -Changing the column type is the only complex action – it changes the contents of files with data. For large tables, this may take a long time. - -Il y a plusieurs étapes de traitement: - -- Préparation de (nouveaux) fichiers temporaires avec des données modifiées. -- Renommer les anciens fichiers. -- Renommer les (nouveaux) fichiers temporaires en anciens noms. -- Suppression des anciens fichiers. - -Seule la première étape prend du temps. Si il y a un échec à ce stade, les données ne sont pas modifiées. -En cas d'échec au cours d'une des étapes successives, les données peuvent être restaurées manuellement. L'exception est si les anciens fichiers ont été supprimés du système de fichiers mais que les données des nouveaux fichiers n'ont pas été écrites sur le disque et ont été perdues. - -Le `ALTER` la requête de modification des colonnes est répliquée. Les instructions sont enregistrées dans ZooKeeper, puis chaque réplique les applique. Tout `ALTER` les requêtes sont exécutées dans le même ordre. La requête attend que les actions appropriées soient terminées sur les autres répliques. Cependant, une requête pour modifier des colonnes dans une table répliquée peut être interrompue, et toutes les actions seront effectuées de manière asynchrone. - -#### Modifier les limites de la requête {#alter-query-limitations} - -Le `ALTER` query vous permet de créer et de supprimer des éléments distincts (colonnes) dans des structures de données imbriquées, mais pas des structures de données imbriquées entières. Pour ajouter une structure de données imbriquée, vous pouvez ajouter des colonnes avec un nom comme `name.nested_name` et le type `Array(T)`. Une structure de données imbriquée est équivalente à plusieurs colonnes de tableau avec un nom qui a le même préfixe avant le point. - -Il n'y a pas de support pour supprimer des colonnes dans la clé primaire ou la clé d'échantillonnage (colonnes qui sont utilisées dans le `ENGINE` expression). La modification du type des colonnes incluses dans la clé primaire n'est possible que si cette modification n'entraîne pas la modification des données (par exemple, vous êtes autorisé à ajouter des valeurs à une énumération ou à modifier un type de `DateTime` de `UInt32`). - -Si l' `ALTER` la requête n'est pas suffisante pour apporter les modifications de table dont vous avez besoin, vous pouvez créer une nouvelle table, y copier les données en utilisant le [INSERT SELECT](insert-into.md#insert_query_insert-select) requête, puis changer les tables en utilisant le [RENAME](misc.md#misc_operations-rename) requête et supprimer l'ancienne table. Vous pouvez utiliser l' [clickhouse-copieur](../../operations/utilities/clickhouse-copier.md) comme une alternative à la `INSERT SELECT` requête. - -Le `ALTER` query bloque toutes les lectures et écritures pour la table. En d'autres termes, si une longue `SELECT` est en cours d'exécution au moment de la `ALTER` requête, la `ALTER` la requête va attendre qu'elle se termine. Dans le même temps, toutes les nouvelles requêtes à la même table attendre que ce `ALTER` est en cours d'exécution. - -Pour les tables qui ne stockent pas les données elles-mêmes (telles que `Merge` et `Distributed`), `ALTER` change simplement la structure de la table, et ne change pas la structure des tables subordonnées. Par exemple, lors de L'exécution de ALTER pour un `Distributed` table, vous devrez également exécuter `ALTER` pour les tables sur tous les serveurs distants. - -### Manipulations avec des Expressions clés {#manipulations-with-key-expressions} - -La commande suivante est prise en charge: - -``` sql -MODIFY ORDER BY new_expression -``` - -Cela ne fonctionne que pour les tables du [`MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) de la famille (y compris les -[répliqué](../../engines/table-engines/mergetree-family/replication.md) table). La commande change l' -[clé de tri](../../engines/table-engines/mergetree-family/mergetree.md) de la table -de `new_expression` (une expression ou un tuple d'expressions). Clé primaire reste le même. - -La commande est légère en ce sens qu'elle ne modifie que les métadonnées. Pour conserver la propriété cette partie de données -les lignes sont ordonnées par l'expression de clé de tri vous ne pouvez pas ajouter d'expressions contenant des colonnes existantes -à la clé de tri (seules les colonnes ajoutées par `ADD COLUMN` commande dans le même `ALTER` requête). - -### Manipulations avec des Indices de saut de données {#manipulations-with-data-skipping-indices} - -Cela ne fonctionne que pour les tables du [`*MergeTree`](../../engines/table-engines/mergetree-family/mergetree.md) de la famille (y compris les -[répliqué](../../engines/table-engines/mergetree-family/replication.md) table). Les opérations suivantes -sont disponibles: - -- `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Ajoute la description de l'index aux métadonnées des tables. - -- `ALTER TABLE [db].name DROP INDEX name` - Supprime la description de l'index des métadonnées des tables et supprime les fichiers d'index du disque. - -Ces commandes sont légères dans le sens où elles ne modifient que les métadonnées ou suppriment des fichiers. -En outre, ils sont répliqués (synchronisation des métadonnées des indices via ZooKeeper). - -### Manipulations avec contraintes {#manipulations-with-constraints} - -En voir plus sur [contraintes](create.md#constraints) - -Les contraintes peuvent être ajoutées ou supprimées à l'aide de la syntaxe suivante: - -``` sql -ALTER TABLE [db].name ADD CONSTRAINT constraint_name CHECK expression; -ALTER TABLE [db].name DROP CONSTRAINT constraint_name; -``` - -Les requêtes ajouteront ou supprimeront des métadonnées sur les contraintes de la table afin qu'elles soient traitées immédiatement. - -Contrainte de vérifier *ne sera pas exécuté* sur les données existantes si elle a été ajoutée. - -Toutes les modifications sur les tables répliquées sont diffusées sur ZooKeeper et seront donc appliquées sur d'autres répliques. - -### Manipulations avec des Partitions et des pièces {#alter_manipulations-with-partitions} - -Les opérations suivantes avec [partition](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) sont disponibles: - -- [DETACH PARTITION](#alter_detach-partition) – Moves a partition to the `detached` répertoire et de l'oublier. -- [DROP PARTITION](#alter_drop-partition) – Deletes a partition. -- [ATTACH PART\|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` répertoire à la table. -- [ATTACH PARTITION FROM](#alter_attach-partition-from) – Copies the data partition from one table to another and adds. -- [REPLACE PARTITION](#alter_replace-partition) - Copie la partition de données d'une table à l'autre et la remplace. -- [MOVE PARTITION TO TABLE](#alter_move_to_table-partition)(#alter_move_to_table-partition) - déplace la partition de données d'une table à l'autre. -- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Rétablit la valeur d'une colonne spécifiée dans une partition. -- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Réinitialise l'index secondaire spécifié dans une partition. -- [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. -- [FETCH PARTITION](#alter_fetch-partition) – Downloads a partition from another server. -- [MOVE PARTITION\|PART](#alter_move-partition) – Move partition/data part to another disk or volume. - - - -#### DETACH PARTITION {#alter_detach-partition} - -``` sql -ALTER TABLE table_name DETACH PARTITION partition_expr -``` - -Déplace toutes les données de la partition spécifiée vers `detached` répertoire. Le serveur oublie la partition de données détachée comme si elle n'existait pas. Le serveur ne connaîtra pas ces données tant que vous n'aurez pas [ATTACH](#alter_attach-partition) requête. - -Exemple: - -``` sql -ALTER TABLE visits DETACH PARTITION 201901 -``` - -Lisez à propos de la définition de l'expression de partition dans une section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -Une fois la requête exécutée, vous pouvez faire ce que vous voulez avec les données du `detached` directory — delete it from the file system, or just leave it. - -This query is replicated – it moves the data to the `detached` répertoire sur toutes les répliques. Notez que vous ne pouvez exécuter cette requête que sur un réplica leader. Pour savoir si une réplique est un leader, effectuez le `SELECT` requête à l' [système.réplique](../../operations/system-tables.md#system_tables-replicas) table. Alternativement, il est plus facile de faire une `DETACH` requête sur toutes les répliques - toutes les répliques lancent une exception, à l'exception de la réplique leader. - -#### DROP PARTITION {#alter_drop-partition} - -``` sql -ALTER TABLE table_name DROP PARTITION partition_expr -``` - -Supprime la partition spécifiée de la table. Cette requête marque la partition comme inactive et supprime complètement les données, environ en 10 minutes. - -Lisez à propos de la définition de l'expression de partition dans une section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -The query is replicated – it deletes data on all replicas. - -#### DROP DETACHED PARTITION\|PART {#alter_drop-detached} - -``` sql -ALTER TABLE table_name DROP DETACHED PARTITION|PART partition_expr -``` - -Supprime la partie spécifiée ou toutes les parties de la partition spécifiée de `detached`. -En savoir plus sur la définition de l'expression de partition dans une section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -#### ATTACH PARTITION\|PART {#alter_attach-partition} - -``` sql -ALTER TABLE table_name ATTACH PARTITION|PART partition_expr -``` - -Ajoute des données à la table à partir du `detached` répertoire. Il est possible d'ajouter des données dans une partition entière ou pour une partie distincte. Exemple: - -``` sql -ALTER TABLE visits ATTACH PARTITION 201901; -ALTER TABLE visits ATTACH PART 201901_2_2_0; -``` - -En savoir plus sur la définition de l'expression de partition dans une section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -Cette requête est répliquée. L'initiateur de réplica vérifie s'il y a des données dans le `detached` répertoire. Si des données existent, la requête vérifie son intégrité. Si tout est correct, la requête ajoute les données à la table. Tous les autres réplicas téléchargent les données de l'initiateur de réplica. - -Ainsi, vous pouvez mettre des données à la `detached` répertoire sur une réplique, et utilisez le `ALTER ... ATTACH` requête pour l'ajouter à la table sur tous les réplicas. - -#### ATTACH PARTITION FROM {#alter_attach-partition-from} - -``` sql -ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1 -``` - -Cette requête copie la partition de données du `table1` de `table2` ajoute des données de gratuit dans la `table2`. Notez que les données ne seront pas supprimées de `table1`. - -Pour que la requête s'exécute correctement, les conditions suivantes doivent être remplies: - -- Les deux tables doivent avoir la même structure. -- Les deux tables doivent avoir la même clé de partition. - -#### REPLACE PARTITION {#alter_replace-partition} - -``` sql -ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 -``` - -Cette requête copie la partition de données du `table1` de `table2` et remplace la partition existante dans le `table2`. Notez que les données ne seront pas supprimées de `table1`. - -Pour que la requête s'exécute correctement, les conditions suivantes doivent être remplies: - -- Les deux tables doivent avoir la même structure. -- Les deux tables doivent avoir la même clé de partition. - -#### MOVE PARTITION TO TABLE {#alter_move_to_table-partition} - -``` sql -ALTER TABLE table_source MOVE PARTITION partition_expr TO TABLE table_dest -``` - -Cette requête déplace la partition de données du `table_source` de `table_dest` avec la suppression des données de `table_source`. - -Pour que la requête s'exécute correctement, les conditions suivantes doivent être remplies: - -- Les deux tables doivent avoir la même structure. -- Les deux tables doivent avoir la même clé de partition. -- Les deux tables doivent appartenir à la même famille de moteurs. (répliqué ou non répliqué) -- Les deux tables doivent avoir la même stratégie de stockage. - -#### CLEAR COLUMN IN PARTITION {#alter_clear-column-partition} - -``` sql -ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr -``` - -Réinitialise toutes les valeurs de la colonne spécifiée dans une partition. Si l' `DEFAULT` la clause a été déterminée lors de la création d'une table, cette requête définit la valeur de la colonne à une valeur par défaut spécifiée. - -Exemple: - -``` sql -ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 -``` - -#### FREEZE PARTITION {#alter_freeze-partition} - -``` sql -ALTER TABLE table_name FREEZE [PARTITION partition_expr] -``` - -Cette requête crée une sauvegarde locale d'une partition spécifiée. Si l' `PARTITION` la clause est omise, la requête crée la sauvegarde de toutes les partitions à la fois. - -!!! note "Note" - L'ensemble du processus de sauvegarde est effectuée sans arrêter le serveur. - -Notez que pour les tables de style ancien, vous pouvez spécifier le préfixe du nom de la partition (par exemple, ‘2019’)- ensuite, la requête crée la sauvegarde pour toutes les partitions correspondantes. Lisez à propos de la définition de l'expression de partition dans une section [Comment spécifier l'expression de partition](#alter-how-to-specify-part-expr). - -Au moment de l'exécution, pour un instantané de données, la requête crée des liens rigides vers des données de table. Les liens sont placés dans le répertoire `/var/lib/clickhouse/shadow/N/...`, où: - -- `/var/lib/clickhouse/` est le répertoire de travail clickhouse spécifié dans la configuration. -- `N` est le numéro incrémental de la sauvegarde. - -!!! note "Note" - Si vous utilisez [un ensemble de disques pour le stockage des données dans une table](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), le `shadow/N` le répertoire apparaît sur chaque disque, stockant les parties de données correspondant `PARTITION` expression. - -La même structure de répertoires est créée à l'intérieur de la sauvegarde qu'à l'intérieur `/var/lib/clickhouse/`. La requête effectue ‘chmod’ pour tous les fichiers, interdisant d'écrire en eux. - -Après avoir créé la sauvegarde, vous pouvez copier les données depuis `/var/lib/clickhouse/shadow/` sur le serveur distant, puis supprimez-le du serveur local. Notez que l' `ALTER t FREEZE PARTITION` la requête n'est pas répliqué. Il crée une sauvegarde locale uniquement sur le serveur local. - -La requête crée une sauvegarde presque instantanément (mais elle attend d'abord que les requêtes en cours à la table correspondante se terminent). - -`ALTER TABLE t FREEZE PARTITION` copie uniquement les données, pas les métadonnées de la table. Faire une sauvegarde des métadonnées de la table, copiez le fichier `/var/lib/clickhouse/metadata/database/table.sql` - -Pour restaurer des données à partir d'une sauvegarde, procédez comme suit: - -1. Créer la table si elle n'existe pas. Pour afficher la requête, utilisez la .fichier sql (remplacer `ATTACH` avec `CREATE`). -2. Copier les données de la `data/database/table/` répertoire à l'intérieur de la sauvegarde `/var/lib/clickhouse/data/database/table/detached/` répertoire. -3. Exécuter `ALTER TABLE t ATTACH PARTITION` les requêtes pour ajouter les données à une table. - -La restauration à partir d'une sauvegarde ne nécessite pas l'arrêt du serveur. - -Pour plus d'informations sur les sauvegardes et la restauration [La Sauvegarde Des Données](../../operations/backup.md) section. - -#### CLEAR INDEX IN PARTITION {#alter_clear-index-partition} - -``` sql -ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr -``` - -La requête fonctionne de manière similaire à `CLEAR COLUMN` mais il remet un index au lieu d'une colonne de données. - -#### FETCH PARTITION {#alter_fetch-partition} - -``` sql -ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'path-in-zookeeper' -``` - -Télécharge une partition depuis un autre serveur. Cette requête ne fonctionne que pour les tables répliquées. - -La requête effectue les opérations suivantes: - -1. Télécharge la partition à partir du fragment spécifié. Dans ‘path-in-zookeeper’ vous devez spécifier un chemin vers le fragment dans ZooKeeper. -2. Ensuite, la requête met les données téléchargées dans le `detached` répertoire de la `table_name` table. L'utilisation de la [ATTACH PARTITION\|PART](#alter_attach-partition) requête pour ajouter les données à la table. - -Exemple: - -``` sql -ALTER TABLE users FETCH PARTITION 201902 FROM '/clickhouse/tables/01-01/visits'; -ALTER TABLE users ATTACH PARTITION 201902; -``` - -Notez que: - -- Le `ALTER ... FETCH PARTITION` la requête n'est pas répliqué. Il place la partition à la `detached` répertoire sur le serveur local. -- Le `ALTER TABLE ... ATTACH` la requête est répliquée. Il ajoute les données à toutes les répliques. Les données sont ajoutées à l'une des répliques `detached` répertoire, et aux autres-des répliques voisines. - -Avant le téléchargement, le système vérifie si la partition existe et si la structure de la table correspond. La réplique la plus appropriée est sélectionnée automatiquement parmi les répliques saines. - -Bien que la requête soit appelée `ALTER TABLE`, il ne modifie pas la structure de la table et ne modifie pas immédiatement les données disponibles dans la table. - -#### MOVE PARTITION\|PART {#alter_move-partition} - -Déplace des partitions ou des parties de données vers un autre volume ou disque pour `MergeTree`-tables de moteur. Voir [Utilisation de plusieurs périphériques de bloc pour le stockage de données](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). - -``` sql -ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' -``` - -Le `ALTER TABLE t MOVE` requête: - -- Non répliqué, car différentes répliques peuvent avoir des stratégies de stockage différentes. -- Renvoie une erreur si le disque ou le volume n'est pas configuré. Query renvoie également une erreur si les conditions de déplacement des données, spécifiées dans la stratégie de stockage, ne peuvent pas être appliquées. -- Peut renvoyer une erreur dans le cas, lorsque les données à déplacer sont déjà déplacées par un processus en arrière-plan, simultané `ALTER TABLE t MOVE` requête ou à la suite de la fusion de données d'arrière-plan. Un utilisateur ne doit effectuer aucune action supplémentaire dans ce cas. - -Exemple: - -``` sql -ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow' -ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' -``` - -#### Comment définir L'Expression de la Partition {#alter-how-to-specify-part-expr} - -Vous pouvez spécifier l'expression de partition dans `ALTER ... PARTITION` requêtes de différentes manières: - -- Comme une valeur de l' `partition` la colonne de la `system.parts` table. Exemple, `ALTER TABLE visits DETACH PARTITION 201901`. -- Comme expression de la colonne de la table. Les constantes et les expressions constantes sont prises en charge. Exemple, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. -- À l'aide de l'ID de partition. Partition ID est un identifiant de chaîne de la partition (lisible par l'homme, si possible) qui est utilisé comme noms de partitions dans le système de fichiers et dans ZooKeeper. L'ID de partition doit être spécifié dans `PARTITION ID` clause, entre guillemets simples. Exemple, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- Dans le [ALTER ATTACH PART](#alter_attach-partition) et [DROP DETACHED PART](#alter_drop-detached) requête, pour spécifier le nom d'une partie, utilisez le littéral de chaîne avec une valeur de `name` la colonne de la [système.detached_parts](../../operations/system-tables.md#system_tables-detached_parts) table. Exemple, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. - -L'utilisation de guillemets lors de la spécification de la partition dépend du type d'expression de partition. Par exemple, pour la `String` type, vous devez spécifier son nom entre guillemets (`'`). Pour l' `Date` et `Int*` types aucune citation n'est nécessaire. - -Pour les tables de style ancien, vous pouvez spécifier la partition sous forme de nombre `201901` ou une chaîne de caractères `'201901'`. La syntaxe des tables new-style est plus stricte avec les types (similaire à l'analyseur pour le format D'entrée des valeurs). - -Toutes les règles ci-dessus sont aussi valables pour la [OPTIMIZE](misc.md#misc_operations-optimize) requête. Si vous devez spécifier la seule partition lors de l'optimisation d'une table non partitionnée, définissez l'expression `PARTITION tuple()`. Exemple: - -``` sql -OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL; -``` - -Les exemples de `ALTER ... PARTITION` les requêtes sont démontrées dans les tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) et [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). - -### Manipulations avec Table TTL {#manipulations-with-table-ttl} - -Vous pouvez modifier [tableau TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) avec une demande du formulaire suivant: - -``` sql -ALTER TABLE table-name MODIFY TTL ttl-expression -``` - -### Synchronicité des requêtes ALTER {#synchronicity-of-alter-queries} - -Pour les tables non réplicables, tous `ALTER` les requêtes sont exécutées simultanément. Pour les tables réplicables, la requête ajoute simplement des instructions pour les actions appropriées à `ZooKeeper` et les actions elles-mêmes sont effectuées dès que possible. Cependant, la requête peut attendre que ces actions soient terminées sur tous les réplicas. - -Pour `ALTER ... ATTACH|DETACH|DROP` les requêtes, vous pouvez utiliser le `replication_alter_partitions_sync` configuration pour configurer l'attente. -Valeurs possibles: `0` – do not wait; `1` – only wait for own execution (default); `2` – wait for all. - -### Mutation {#alter-mutations} - -Les Mutations sont une variante ALTER query qui permet de modifier ou de supprimer des lignes dans une table. Contrairement à la norme `UPDATE` et `DELETE` les requêtes qui sont destinées aux changements de données de point, les mutations sont destinées aux opérations lourdes qui modifient beaucoup de lignes dans une table. Pris en charge pour le `MergeTree` famille de moteurs de table, y compris les moteurs avec support de réplication. - -Les tables existantes sont prêtes pour les mutations telles quelles (aucune conversion nécessaire), mais après l'application de la première mutation à une table, son format de métadonnées devient incompatible avec les versions précédentes du serveur et il devient impossible de revenir à une version précédente. - -Commandes actuellement disponibles: - -``` sql -ALTER TABLE [db.]table DELETE WHERE filter_expr -``` - -Le `filter_expr` doit être de type `UInt8`. La requête supprime les lignes de la table pour lesquelles cette expression prend une valeur différente de zéro. - -``` sql -ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr -``` - -Le `filter_expr` doit être de type `UInt8`. Cette requête met à jour les valeurs des colonnes spécifiées en les valeurs des expressions correspondantes dans les lignes pour lesquelles `filter_expr` prend une valeur non nulle. Les valeurs sont converties en type de colonne à l'aide `CAST` opérateur. La mise à jour des colonnes utilisées dans le calcul de la clé primaire ou de la clé de partition n'est pas prise en charge. - -``` sql -ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name -``` - -La requête reconstruit l'index secondaire `name` dans la partition `partition_name`. - -Une requête peut contenir plusieurs commandes séparées par des virgules. - -Pour les tables \* MergeTree, les mutations s'exécutent en réécrivant des parties de données entières. Il n'y a pas d'atomicité-les pièces sont substituées aux pièces mutées dès qu'elles sont prêtes et un `SELECT` la requête qui a commencé à s'exécuter pendant une mutation verra les données des parties qui ont déjà été mutées ainsi que les données des parties qui n'ont pas encore été mutées. - -Les Mutations sont totalement ordonnées par leur ordre de création et sont appliquées à chaque partie dans cet ordre. Les Mutations sont également partiellement ordonnées avec des insertions - les données insérées dans la table avant la soumission de la mutation seront mutées et les données insérées après ne seront pas mutées. Notez que les mutations ne bloquent en aucune façon les INSERTs. - -Une requête de mutation retourne immédiatement après l'ajout de l'entrée de mutation (dans le cas de tables répliquées à ZooKeeper, pour les tables non compliquées - au système de fichiers). La mutation elle-même s'exécute de manière asynchrone en utilisant les paramètres du profil système. Pour suivre l'avancement des mutations vous pouvez utiliser la [`system.mutations`](../../operations/system-tables.md#system_tables-mutations) table. Une mutation qui a été soumise avec succès continuera à s'exécuter même si les serveurs ClickHouse sont redémarrés. Il n'y a aucun moyen de faire reculer la mutation une fois qu'elle est soumise, mais si la mutation est bloquée pour une raison quelconque, elle peut être annulée avec le [`KILL MUTATION`](misc.md#kill-mutation) requête. - -Les entrées pour les mutations finies ne sont pas supprimées immédiatement (le nombre d'entrées conservées est déterminé par `finished_mutations_to_keep` le moteur de stockage de paramètre). Les anciennes entrées de mutation sont supprimées. - -## ALTER USER {#alter-user-statement} - -Changements clickhouse comptes d'utilisateurs. - -### Syntaxe {#alter-user-syntax} - -``` sql -ALTER USER [IF EXISTS] name [ON CLUSTER cluster_name] - [RENAME TO new_name] - [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}] - [[ADD|DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -### Description {#alter-user-dscr} - -Utiliser `ALTER USER` vous devez avoir le [ALTER USER](grant.md#grant-access-management) privilège. - -### Exemple {#alter-user-examples} - -Définir les rôles accordés par défaut: - -``` sql -ALTER USER user DEFAULT ROLE role1, role2 -``` - -Si les rôles ne sont pas précédemment accordés à un utilisateur, ClickHouse lève une exception. - -Définissez tous les rôles accordés à défaut: - -``` sql -ALTER USER user DEFAULT ROLE ALL -``` - -Si un rôle seront accordés à un utilisateur dans l'avenir, il deviendra automatiquement par défaut. - -Définissez tous les rôles accordés sur default excepting `role1` et `role2`: - -``` sql -ALTER USER user DEFAULT ROLE ALL EXCEPT role1, role2 -``` - -## ALTER ROLE {#alter-role-statement} - -Les changements de rôles. - -### Syntaxe {#alter-role-syntax} - -``` sql -ALTER ROLE [IF EXISTS] name [ON CLUSTER cluster_name] - [RENAME TO new_name] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -## ALTER ROW POLICY {#alter-row-policy-statement} - -Modifie la stratégie de ligne. - -### Syntaxe {#alter-row-policy-syntax} - -``` sql -ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]table - [RENAME TO new_name] - [AS {PERMISSIVE | RESTRICTIVE}] - [FOR SELECT] - [USING {condition | NONE}][,...] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -## ALTER QUOTA {#alter-quota-statement} - -Les changements de quotas. - -### Syntaxe {#alter-quota-syntax} - -``` sql -ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] - [RENAME TO new_name] - [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} - {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | - NO LIMITS | TRACKING ONLY} [,...]] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -## ALTER SETTINGS PROFILE {#alter-settings-profile-statement} - -Les changements de quotas. - -### Syntaxe {#alter-settings-profile-syntax} - -``` sql -ALTER SETTINGS PROFILE [IF EXISTS] name [ON CLUSTER cluster_name] - [RENAME TO new_name] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...] -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/alter/) diff --git a/docs/fr/sql-reference/statements/create.md b/docs/fr/sql-reference/statements/create.md deleted file mode 100644 index e7c8040ee6e..00000000000 --- a/docs/fr/sql-reference/statements/create.md +++ /dev/null @@ -1,502 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 35 -toc_title: CREATE ---- - -# Créer des requêtes {#create-queries} - -## CREATE DATABASE {#query-language-create-database} - -Crée la base de données. - -``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] -``` - -### Clause {#clauses} - -- `IF NOT EXISTS` - Si l' `db_name` la base de données existe déjà, alors ClickHouse ne crée pas de nouvelle base de données et: - - - Ne lance pas d'exception si la clause est spécifiée. - - Lève une exception si la clause n'est pas spécifiée. - -- `ON CLUSTER` - Clickhouse crée le `db_name` base de données sur tous les serveurs d'un cluster spécifié. - -- `ENGINE` - - - [MySQL](../../engines/database-engines/mysql.md) - Vous permet de récupérer des données à partir du serveur MySQL distant. - Par défaut, ClickHouse utilise son propre [moteur de base de données](../../engines/database-engines/index.md). - -## CREATE TABLE {#create-table-query} - -Le `CREATE TABLE` la requête peut avoir plusieurs formes. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec] [TTL expr2], - ... -) ENGINE = engine -``` - -Crée une table nommée ‘name’ dans le ‘db’ base de données ou la base de données actuelle si ‘db’ n'est pas définie, avec la structure spécifiée entre parenthèses et l' ‘engine’ moteur. -La structure de la table est une liste de descriptions de colonnes. Si les index sont pris en charge par le moteur, ils sont indiqués comme paramètres pour le moteur de table. - -Une description de colonne est `name type` dans le cas le plus simple. Exemple: `RegionID UInt32`. -Des Expressions peuvent également être définies pour les valeurs par défaut (voir ci-dessous). - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] -``` - -Crée une table avec la même structure qu'une autre table. Vous pouvez spécifier un moteur différent pour la table. Si le moteur n'est pas spécifié, le même moteur sera utilisé que pour la `db2.name2` table. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name AS table_function() -``` - -Crée une table avec la structure et les données renvoyées par [fonction de table](../table-functions/index.md#table-functions). - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... -``` - -Crée une table avec une structure comme le résultat de l' `SELECT` une requête avec les ‘engine’ moteur, et le remplit avec des données de SELECT. - -Dans tous les cas, si `IF NOT EXISTS` est spécifié, la requête ne renvoie pas une erreur si la table existe déjà. Dans ce cas, la requête ne font rien. - -Il peut y avoir d'autres clauses après le `ENGINE` la clause dans la requête. Voir la documentation détaillée sur la façon de créer des tables dans les descriptions de [moteurs de table](../../engines/table-engines/index.md#table_engines). - -### Les Valeurs Par Défaut {#create-default-values} - -La description de colonne peut spécifier une expression pour une valeur par défaut, de l'une des manières suivantes:`DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. -Exemple: `URLDomain String DEFAULT domain(URL)`. - -Si une expression pour la valeur par défaut n'est pas définie, les valeurs par défaut seront définies sur zéros pour les nombres, chaînes vides pour les chaînes, tableaux vides pour les tableaux et `1970-01-01` pour les dates ou zero unix timestamp pour les dates avec le temps. Les valeurs NULL ne sont pas prises en charge. - -Si l'expression par défaut est définie, le type de colonne est facultatif. S'il n'y a pas de type explicitement défini, le type d'expression par défaut est utilisé. Exemple: `EventDate DEFAULT toDate(EventTime)` – the ‘Date’ type sera utilisé pour la ‘EventDate’ colonne. - -Si le type de données et l'expression par défaut sont définis explicitement, cette expression sera convertie au type spécifié à l'aide des fonctions de conversion de type. Exemple: `Hits UInt32 DEFAULT 0` signifie la même chose que `Hits UInt32 DEFAULT toUInt32(0)`. - -Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions don't contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. - -`DEFAULT expr` - -Valeur par défaut normale. Si la requête INSERT ne spécifie pas la colonne correspondante, elle sera remplie en calculant l'expression correspondante. - -`MATERIALIZED expr` - -Expression matérialisée. Une telle colonne ne peut pas être spécifiée pour INSERT, car elle est toujours calculée. -Pour un INSERT sans Liste de colonnes, ces colonnes ne sont pas prises en compte. -De plus, cette colonne n'est pas substituée lors de l'utilisation d'un astérisque dans une requête SELECT. C'est pour préserver l'invariant que le dump obtenu en utilisant `SELECT *` peut être inséré dans la table en utilisant INSERT sans spécifier la liste des colonnes. - -`ALIAS expr` - -Synonyme. Une telle colonne n'est pas du tout stockée dans la table. -Ses valeurs ne peuvent pas être insérées dans une table et elles ne sont pas substituées lors de l'utilisation d'un astérisque dans une requête SELECT. -Il peut être utilisé dans SELECTs si l'alias est développé pendant l'analyse des requêtes. - -Lorsque vous utilisez la requête ALTER pour ajouter de nouvelles colonnes, les anciennes données de ces colonnes ne sont pas écrites. Au lieu de cela, lors de la lecture d'anciennes données qui n'ont pas de valeurs pour les nouvelles colonnes, les expressions sont calculées à la volée par défaut. Cependant, si l'exécution des expressions nécessite différentes colonnes qui ne sont pas indiquées dans la requête, ces colonnes seront en outre lues, mais uniquement pour les blocs de données qui en ont besoin. - -Si vous ajoutez une nouvelle colonne à une table mais modifiez ultérieurement son expression par défaut, les valeurs utilisées pour les anciennes données changeront (pour les données où les valeurs n'ont pas été stockées sur le disque). Notez que lors de l'exécution de fusions d'arrière-plan, les données des colonnes manquantes dans l'une des parties de fusion sont écrites dans la partie fusionnée. - -Il n'est pas possible de définir des valeurs par défaut pour les éléments dans les structures de données. - -### Contraintes {#constraints} - -Avec les descriptions de colonnes des contraintes peuvent être définies: - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], - ... - CONSTRAINT constraint_name_1 CHECK boolean_expr_1, - ... -) ENGINE = engine -``` - -`boolean_expr_1` pourrait par n'importe quelle expression booléenne. Si les contraintes sont définies pour la table, chacun d'eux sera vérifiée pour chaque ligne `INSERT` query. If any constraint is not satisfied — server will raise an exception with constraint name and checking expression. - -L'ajout d'une grande quantité de contraintes peut affecter négativement les performances de big `INSERT` requête. - -### Expression TTL {#ttl-expression} - -Définit la durée de stockage des valeurs. Peut être spécifié uniquement pour les tables mergetree-family. Pour la description détaillée, voir [TTL pour les colonnes et les tableaux](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). - -### Codecs De Compression De Colonne {#codecs} - -Par défaut, ClickHouse applique le `lz4` méthode de compression. Pour `MergeTree`- famille de moteurs Vous pouvez modifier la méthode de compression par défaut dans le [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) section d'une configuration de serveur. Vous pouvez également définir la méthode de compression pour chaque colonne `CREATE TABLE` requête. - -``` sql -CREATE TABLE codec_example -( - dt Date CODEC(ZSTD), - ts DateTime CODEC(LZ4HC), - float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(9)) - value Float32 CODEC(Delta, ZSTD) -) -ENGINE = -... -``` - -Si un codec est spécifié, le codec par défaut ne s'applique pas. Les Codecs peuvent être combinés dans un pipeline, par exemple, `CODEC(Delta, ZSTD)`. Pour sélectionner la meilleure combinaison de codecs pour votre projet, passez des benchmarks similaires à ceux décrits dans Altinity [Nouveaux encodages pour améliorer L'efficacité du ClickHouse](https://www.altinity.com/blog/2019/7/new-encodings-to-improve-clickhouse) article. - -!!! warning "Avertissement" - Vous ne pouvez pas décompresser les fichiers de base de données ClickHouse avec des utilitaires externes tels que `lz4`. Au lieu de cela, utilisez le spécial [clickhouse-compresseur](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utilitaire. - -La Compression est prise en charge pour les moteurs de tableau suivants: - -- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) famille. Prend en charge les codecs de compression de colonne et la sélection de la méthode de compression par défaut par [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) paramètre. -- [Journal](../../engines/table-engines/log-family/index.md) famille. Utilise le `lz4` méthode de compression par défaut et prend en charge les codecs de compression de colonne. -- [Définir](../../engines/table-engines/special/set.md). Uniquement pris en charge la compression par défaut. -- [Rejoindre](../../engines/table-engines/special/join.md). Uniquement pris en charge la compression par défaut. - -ClickHouse prend en charge les codecs à usage commun et les codecs spécialisés. - -#### Codecs Spécialisés {#create-query-specialized-codecs} - -Ces codecs sont conçus pour rendre la compression plus efficace en utilisant des fonctionnalités spécifiques des données. Certains de ces codecs ne compressent pas les données eux-mêmes. Au lieu de cela, ils préparent les données pour un codec à usage commun, qui les compresse mieux que sans cette préparation. - -Spécialisé codecs: - -- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` sont utilisés pour stocker des valeurs delta, donc `delta_bytes` est la taille maximale des valeurs brutes. Possible `delta_bytes` valeurs: 1, 2, 4, 8. La valeur par défaut pour `delta_bytes` être `sizeof(type)` si égale à 1, 2, 4 ou 8. Dans tous les autres cas, c'est 1. -- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: Une Base De Données De Séries Chronologiques Rapide, Évolutive Et En Mémoire](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). -- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: Une Base De Données De Séries Chronologiques Rapide, Évolutive Et En Mémoire](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). -- `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` et `DateTime`). À chaque étape de son algorithme, le codec prend un bloc de 64 valeurs, les place dans une matrice de 64x64 bits, le transpose, recadre les bits de valeurs inutilisés et renvoie le reste sous forme de séquence. Les bits inutilisés sont les bits, qui ne diffèrent pas entre les valeurs maximum et minimum dans la partie de données entière pour laquelle la compression est utilisée. - -`DoubleDelta` et `Gorilla` les codecs sont utilisés dans Gorilla TSDB comme composants de son algorithme de compression. L'approche Gorilla est efficace dans les scénarios où il y a une séquence de valeurs qui changent lentement avec leurs horodatages. Les horodatages sont effectivement compressés par le `DoubleDelta` codec, et les valeurs sont effectivement comprimé par le `Gorilla` codec. Par exemple, pour obtenir une table stockée efficacement, vous pouvez la créer dans la configuration suivante: - -``` sql -CREATE TABLE codec_example -( - timestamp DateTime CODEC(DoubleDelta), - slow_values Float32 CODEC(Gorilla) -) -ENGINE = MergeTree() -``` - -#### Codecs À Usage Général {#create-query-general-purpose-codecs} - -Codec: - -- `NONE` — No compression. -- `LZ4` — Lossless [algorithme de compression de données](https://github.com/lz4/lz4) utilisé par défaut. Applique la compression rapide LZ4. -- `LZ4HC[(level)]` — LZ4 HC (high compression) algorithm with configurable level. Default level: 9. Setting `level <= 0` s'applique le niveau par défaut. Niveaux possibles: \[1, 12\]. Plage de niveau recommandée: \[4, 9\]. -- `ZSTD[(level)]` — [Algorithme de compression ZSTD](https://en.wikipedia.org/wiki/Zstandard) avec configurables `level`. Niveaux possibles: \[1, 22\]. Valeur par défaut: 1. - -Des niveaux de compression élevés sont utiles pour les scénarios asymétriques, comme compresser une fois, décompresser à plusieurs reprises. Des niveaux plus élevés signifient une meilleure compression et une utilisation plus élevée du processeur. - -## Les Tables Temporaires {#temporary-tables} - -Clickhouse prend en charge les tables temporaires qui ont les caractéristiques suivantes: - -- Les tables temporaires disparaissent à la fin de la session, y compris si la connexion est perdue. -- Une table temporaire utilise uniquement le moteur de mémoire. -- La base de données ne peut pas être spécifiée pour une table temporaire. Il est créé en dehors des bases de données. -- Impossible de créer une table temporaire avec une requête DDL distribuée sur tous les serveurs de cluster (en utilisant `ON CLUSTER`): ce tableau n'existe que dans la session en cours. -- Si une table temporaire a le même nom qu'une autre et qu'une requête spécifie le nom de la table sans spécifier la base de données, la table temporaire sera utilisée. -- Pour le traitement des requêtes distribuées, les tables temporaires utilisées dans une requête sont transmises à des serveurs distants. - -Pour créer une table temporaire, utilisez la syntaxe suivante: - -``` sql -CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) -``` - -Dans la plupart des cas, les tables temporaires ne sont pas créées manuellement, mais lors de l'utilisation de données externes pour une requête ou pour `(GLOBAL) IN`. Pour plus d'informations, consultez les sections appropriées - -Il est possible d'utiliser des tables avec [Moteur = mémoire](../../engines/table-engines/special/memory.md) au lieu de tables temporaires. - -## Requêtes DDL distribuées (sur la clause CLUSTER) {#distributed-ddl-queries-on-cluster-clause} - -Le `CREATE`, `DROP`, `ALTER`, et `RENAME` les requêtes prennent en charge l'exécution distribuée sur un cluster. -Par exemple, la requête suivante crée la `all_hits` `Distributed` tableau sur chaque ordinateur hôte `cluster`: - -``` sql -CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) -``` - -Pour exécuter ces requêtes correctement, chaque hôte doit avoir la même définition de cluster (pour simplifier la synchronisation des configs, vous pouvez utiliser des substitutions de ZooKeeper). Ils doivent également se connecter aux serveurs ZooKeeper. -La version locale de la requête sera finalement implémentée sur chaque hôte du cluster, même si certains hôtes ne sont actuellement pas disponibles. L'ordre d'exécution des requêtes au sein d'un seul hôte est garanti. - -## CREATE VIEW {#create-view} - -``` sql -CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]table_name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... -``` - -Crée une vue. Il existe deux types de vues: normale et matérialisée. - -Les vues normales ne stockent aucune donnée, mais effectuent simplement une lecture à partir d'une autre table. En d'autres termes, une vue normale n'est rien de plus qu'une requête enregistrée. Lors de la lecture à partir d'une vue, cette requête enregistrée est utilisée comme sous-requête dans la clause FROM. - -Par exemple, supposons que vous avez créé une vue: - -``` sql -CREATE VIEW view AS SELECT ... -``` - -et écrit une requête: - -``` sql -SELECT a, b, c FROM view -``` - -Cette requête est entièrement équivalente à l'utilisation de la sous requête: - -``` sql -SELECT a, b, c FROM (SELECT ...) -``` - -Les vues matérialisées stockent les données transformées par la requête SELECT correspondante. - -Lors de la création d'une vue matérialisée sans `TO [db].[table]`, you must specify ENGINE – the table engine for storing data. - -Lors de la création d'une vue matérialisée avec `TO [db].[table]` vous ne devez pas utiliser `POPULATE`. - -Une vue matérialisée est agencée comme suit: lors de l'insertion de données dans la table spécifiée dans SELECT, une partie des données insérées est convertie par cette requête SELECT, et le résultat est inséré dans la vue. - -Si vous spécifiez POPULATE, les données de table existantes sont insérées dans la vue lors de sa création, comme si `CREATE TABLE ... AS SELECT ...` . Sinon, la requête ne contient que les données insérées dans la table après la création de la vue. Nous ne recommandons pas D'utiliser POPULATE, car les données insérées dans la table lors de la création de la vue ne seront pas insérées dedans. - -A `SELECT` la requête peut contenir `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` est définie, les données sont agrégées lors de l'insertion, mais uniquement dans un seul paquet de données insérées. Les données ne seront pas agrégées davantage. L'exception concerne l'utilisation d'un moteur qui effectue indépendamment l'agrégation de données, par exemple `SummingMergeTree`. - -L'exécution de `ALTER` les requêtes sur les vues matérialisées n'ont pas été complètement développées, elles pourraient donc être gênantes. Si la vue matérialisée utilise la construction `TO [db.]name` vous pouvez `DETACH` la vue, exécutez `ALTER` pour la table cible, puis `ATTACH` précédemment détaché (`DETACH`) vue. - -Les vues ressemblent aux tables normales. Par exemple, ils sont répertoriés dans le résultat de la `SHOW TABLES` requête. - -Il n'y a pas de requête séparée pour supprimer des vues. Pour supprimer une vue, utilisez `DROP TABLE`. - -## CREATE DICTIONARY {#create-dictionary-query} - -``` sql -CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] -( - key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - attr1 type2 [DEFAULT|EXPRESSION expr3], - attr2 type2 [DEFAULT|EXPRESSION expr4] -) -PRIMARY KEY key1, key2 -SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN])) -LAYOUT(LAYOUT_NAME([param_name param_value])) -LIFETIME({MIN min_val MAX max_val | max_val}) -``` - -Crée [externe dictionnaire](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) avec le [structure](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [disposition](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) et [vie](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). - -Structure de dictionnaire externe se compose d'attributs. Les attributs du dictionnaire sont spécifiés de la même manière que les colonnes du tableau. La seule propriété d'attribut requise est son type, toutes les autres propriétés peuvent avoir des valeurs par défaut. - -Selon le dictionnaire [disposition](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) un ou plusieurs attributs peuvent être spécifiés comme les clés de dictionnaire. - -Pour plus d'informations, voir [Dictionnaires Externes](../dictionaries/external-dictionaries/external-dicts.md) section. - -## CREATE USER {#create-user-statement} - -Crée un [compte d'utilisateur](../../operations/access-rights.md#user-account-management). - -### Syntaxe {#create-user-syntax} - -``` sql -CREATE USER [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] - [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - [DEFAULT ROLE role [,...]] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -#### Identification {#identification} - -Il existe de multiples façons d'identification d'un utilisateur: - -- `IDENTIFIED WITH no_password` -- `IDENTIFIED WITH plaintext_password BY 'qwerty'` -- `IDENTIFIED WITH sha256_password BY 'qwerty'` ou `IDENTIFIED BY 'password'` -- `IDENTIFIED WITH sha256_hash BY 'hash'` -- `IDENTIFIED WITH double_sha1_password BY 'qwerty'` -- `IDENTIFIED WITH double_sha1_hash BY 'hash'` - -#### L'Utilisateur De L'Hôte {#user-host} - -L'hôte utilisateur est un hôte à partir duquel une connexion au serveur ClickHouse peut être établie. Hôte peut être spécifié dans le `HOST` section de requête par les moyens suivants: - -- `HOST IP 'ip_address_or_subnetwork'` — User can connect to ClickHouse server only from the specified IP address or a [sous-réseau](https://en.wikipedia.org/wiki/Subnetwork). Exemple: `HOST IP '192.168.0.0/16'`, `HOST IP '2001:DB8::/32'`. Pour une utilisation en production, spécifiez uniquement `HOST IP` (adresses IP et leurs masques), depuis l'utilisation `host` et `host_regexp` peut causer une latence supplémentaire. -- `HOST ANY` — User can connect from any location. This is default option. -- `HOST LOCAL` — User can connect only locally. -- `HOST NAME 'fqdn'` — User host can be specified as FQDN. For example, `HOST NAME 'mysite.com'`. -- `HOST NAME REGEXP 'regexp'` — You can use [pcre](http://www.pcre.org/) expressions régulières lors de la spécification des hôtes utilisateur. Exemple, `HOST NAME REGEXP '.*\.mysite\.com'`. -- `HOST LIKE 'template'` — Allows you use the [LIKE](../functions/string-search-functions.md#function-like) opérateur de filtre de l'utilisateur hôtes. Exemple, `HOST LIKE '%'` est équivalent à `HOST ANY`, `HOST LIKE '%.mysite.com'` filtre tous les hôtes dans le `mysite.com` domaine. - -Une autre façon de spécifier l'hôte est d'utiliser `@` syntaxe avec le nom d'utilisateur. Exemple: - -- `CREATE USER mira@'127.0.0.1'` — Equivalent to the `HOST IP` syntaxe. -- `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntaxe. -- `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntaxe. - -!!! info "Avertissement" - Clickhouse traite `user_name@'address'` comme un nom d'utilisateur dans son ensemble. Donc, techniquement, vous pouvez créer plusieurs utilisateurs avec `user_name` et différentes constructions après `@`. Nous ne recommandons pas de le faire. - -### Exemple {#create-user-examples} - -Créer le compte d'utilisateur `mira` protégé par le mot de passe `qwerty`: - -``` sql -CREATE USER mira HOST IP '127.0.0.1' IDENTIFIED WITH sha256_password BY 'qwerty' -``` - -`mira` devrait démarrer l'application client sur l'hôte où le serveur ClickHouse s'exécute. - -Créer le compte d'utilisateur `john`, attribuez-lui des rôles et définissez ces rôles par défaut: - -``` sql -CREATE USER john DEFAULT ROLE role1, role2 -``` - -Créer le compte d'utilisateur `john` et faire tous ses futurs rôles par défaut: - -``` sql -ALTER USER user DEFAULT ROLE ALL -``` - -Quand un rôle sera attribué à `john` dans l'avenir, il deviendra automatiquement par défaut. - -Créer le compte d'utilisateur `john` et faire tous ses futurs rôles par défaut sauf `role1` et `role2`: - -``` sql -ALTER USER john DEFAULT ROLE ALL EXCEPT role1, role2 -``` - -## CREATE ROLE {#create-role-statement} - -Crée un [rôle](../../operations/access-rights.md#role-management). - -### Syntaxe {#create-role-syntax} - -``` sql -CREATE ROLE [IF NOT EXISTS | OR REPLACE] name - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -### Description {#create-role-description} - -Rôle est un ensemble de [privilège](grant.md#grant-privileges). Un utilisateur reçoit un rôle obtient tous les privilèges de ce rôle. - -Un utilisateur peut être affecté à plusieurs rôles. Les utilisateurs peuvent appliquer leurs rôles accordés dans des combinaisons arbitraires par le [SET ROLE](misc.md#set-role-statement) déclaration. La finale de la portée des privilèges est un ensemble combiné de tous les privilèges de tous les rôles. Si un utilisateur a des privilèges accordés directement à son compte d'utilisateur, ils sont également combinés avec les privilèges accordés par les rôles. - -L'utilisateur peut avoir des rôles par défaut qui s'appliquent à la connexion de l'utilisateur. Pour définir les rôles par défaut, utilisez [SET DEFAULT ROLE](misc.md#set-default-role-statement) - déclaration ou de la [ALTER USER](alter.md#alter-user-statement) déclaration. - -Pour révoquer un rôle, utilisez [REVOKE](revoke.md) déclaration. - -Pour supprimer le rôle, utilisez [DROP ROLE](misc.md#drop-role-statement) déclaration. Le rôle supprimé est automatiquement révoqué de tous les utilisateurs et rôles auxquels il a été accordé. - -### Exemple {#create-role-examples} - -``` sql -CREATE ROLE accountant; -GRANT SELECT ON db.* TO accountant; -``` - -Cette séquence de requêtes crée le rôle `accountant` cela a le privilège de lire les données du `accounting` la base de données. - -Octroi du rôle à l'utilisateur `mira`: - -``` sql -GRANT accountant TO mira; -``` - -Une fois le rôle accordé, l'utilisateur peut l'utiliser et effectuer les requêtes autorisées. Exemple: - -``` sql -SET ROLE accountant; -SELECT * FROM db.*; -``` - -## CREATE ROW POLICY {#create-row-policy-statement} - -Crée un [filtre pour les lignes](../../operations/access-rights.md#row-policy-management) qu'un utilisateur peut lire à partir d'une table. - -### Syntaxe {#create-row-policy-syntax} - -``` sql -CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster_name] ON [db.]table - [AS {PERMISSIVE | RESTRICTIVE}] - [FOR SELECT] - [USING condition] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -#### Section AS {#create-row-policy-as} - -À l'aide de cette section, vous pouvez créer des stratégies permissives ou restrictives. - -La stratégie Permissive accorde l'accès aux lignes. Les stratégies permissives qui s'appliquent à la même table sont combinées ensemble en utilisant le booléen `OR` opérateur. Les stratégies sont permissives par défaut. - -La politique Restrictive limite l'accès à la ligne. Les politiques restrictives qui s'appliquent à la même table sont combinées en utilisant le booléen `AND` opérateur. - -Les stratégies restrictives s'appliquent aux lignes qui ont passé les filtres permissifs. Si vous définissez des stratégies restrictives mais aucune politique permissive, l'utilisateur ne peut obtenir aucune ligne de la table. - -#### La Section DE {#create-row-policy-to} - -Dans la section `TO` vous pouvez donner une liste mixte de rôles et d'utilisateurs, par exemple, `CREATE ROW POLICY ... TO accountant, john@localhost`. - -Mot `ALL` signifie Tous les utilisateurs de ClickHouse, y compris l'utilisateur actuel. Mot `ALL EXCEPT` autoriser à exclure certains utilisateurs de la liste tous les utilisateurs, par exemple `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` - -### Exemple {#examples} - -- `CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO accountant, john@localhost` -- `CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO ALL EXCEPT mira` - -## CREATE QUOTA {#create-quota-statement} - -Crée un [quota](../../operations/access-rights.md#quotas-management) qui peut être attribué à un utilisateur ou un rôle. - -### Syntaxe {#create-quota-syntax} - -``` sql -CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} - {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | - NO LIMITS | TRACKING ONLY} [,...]] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -### Exemple {#create-quota-example} - -Limiter le nombre maximum de requêtes pour l'utilisateur actuel avec 123 requêtes en 15 mois contrainte: - -``` sql -CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER -``` - -## CREATE SETTINGS PROFILE {#create-settings-profile-statement} - -Crée un [les paramètres de profil](../../operations/access-rights.md#settings-profiles-management) qui peut être attribué à un utilisateur ou un rôle. - -### Syntaxe {#create-settings-profile-syntax} - -``` sql -CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...] -``` - -# Exemple {#create-settings-profile-syntax} - -Créer l' `max_memory_usage_profile` paramètres du profil avec valeur et contraintes pour `max_memory_usage` paramètre. L'affecter à `robin`: - -``` sql -CREATE SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/create/) diff --git a/docs/fr/sql-reference/statements/grant.md b/docs/fr/sql-reference/statements/grant.md deleted file mode 100644 index 143c9a36e33..00000000000 --- a/docs/fr/sql-reference/statements/grant.md +++ /dev/null @@ -1,476 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: GRANT ---- - -# GRANT {#grant} - -- Accorder [privilège](#grant-privileges) pour ClickHouse comptes d'utilisateurs ou des rôles. -- Affecte des rôles à des comptes d'utilisateurs ou à d'autres rôles. - -Pour révoquer les privilèges, utilisez [REVOKE](revoke.md) déclaration. Vous pouvez également classer les privilèges accordés par le [SHOW GRANTS](show.md#show-grants-statement) déclaration. - -## Accorder La Syntaxe Des Privilèges {#grant-privigele-syntax} - -``` sql -GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user | role | CURRENT_USER} [,...] [WITH GRANT OPTION] -``` - -- `privilege` — Type of privilege. -- `role` — ClickHouse user role. -- `user` — ClickHouse user account. - -Le `WITH GRANT OPTION` clause de subventions `user` ou `role` avec l'autorisation de réaliser des `GRANT` requête. Les utilisateurs peuvent accorder des privilèges de la même portée qu'ils ont et moins. - -## Attribution De La Syntaxe Du Rôle {#assign-role-syntax} - -``` sql -GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_USER} [,...] [WITH ADMIN OPTION] -``` - -- `role` — ClickHouse user role. -- `user` — ClickHouse user account. - -Le `WITH ADMIN OPTION` clause de jeux [ADMIN OPTION](#admin-option-privilege) privilège pour `user` ou `role`. - -## Utilisation {#grant-usage} - -Utiliser `GRANT` votre compte doit avoir le `GRANT OPTION` privilège. Vous ne pouvez accorder des privilèges que dans le cadre de vos privilèges de Compte. - -Par exemple, l'administrateur a accordé des privilèges `john` compte par la requête: - -``` sql -GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION -``` - -Cela signifie que `john` a la permission d'effectuer: - -- `SELECT x,y FROM db.table`. -- `SELECT x FROM db.table`. -- `SELECT y FROM db.table`. - -`john` ne pouvez pas effectuer de `SELECT z FROM db.table`. Le `SELECT * FROM db.table` aussi n'est pas disponible. En traitant cette requête, ClickHouse ne renvoie aucune donnée, même `x` et `y`. La seule exception est si une table contient uniquement `x` et `y` colonnes, dans ce cas ClickHouse renvoie toutes les données. - -Également `john` a l' `GRANT OPTION` privilège, de sorte qu'il peut accorder à d'autres utilisateurs avec des privilèges de la même ou de la plus petite portée. - -Spécification des privilèges vous pouvez utiliser asterisk (`*`) au lieu d'une table ou d'un nom de base de données. Par exemple, l' `GRANT SELECT ON db.* TO john` requête permet `john` pour effectuer la `SELECT` requête sur toutes les tables dans `db` la base de données. En outre, vous pouvez omettre le nom de la base de données. Dans ce cas, des privilèges sont accordés pour la base de données actuelle, par exemple: `GRANT SELECT ON * TO john` accorde le privilège sur toutes les tables dans la base de données actuelle, `GRANT SELECT ON mytable TO john` accorde le privilège sur le `mytable` table dans la base de données actuelle. - -L'accès à la `system` la base de données est toujours autorisée (puisque cette base de données est utilisée pour traiter les requêtes). - -Vous pouvez accorder plusieurs privilèges à plusieurs comptes dans une requête. Requête `GRANT SELECT, INSERT ON *.* TO john, robin` permet de comptes `john` et `robin` pour effectuer la `INSERT` et `SELECT` des requêtes sur toutes les tables de toutes les bases de données sur le serveur. - -## Privilège {#grant-privileges} - -Privilège est une autorisation pour effectuer un type spécifique de requêtes. - -Les privilèges ont une structure hiérarchique. Un ensemble de requêtes autorisées dépend de la portée des privilèges. - -Hiérarchie des privilèges: - -- [SELECT](#grant-select) -- [INSERT](#grant-insert) -- [ALTER](#grant-alter) - - `ALTER TABLE` - - `ALTER UPDATE` - - `ALTER DELETE` - - `ALTER COLUMN` - - `ALTER ADD COLUMN` - - `ALTER DROP COLUMN` - - `ALTER MODIFY COLUMN` - - `ALTER COMMENT COLUMN` - - `ALTER CLEAR COLUMN` - - `ALTER RENAME COLUMN` - - `ALTER INDEX` - - `ALTER ORDER BY` - - `ALTER ADD INDEX` - - `ALTER DROP INDEX` - - `ALTER MATERIALIZE INDEX` - - `ALTER CLEAR INDEX` - - `ALTER CONSTRAINT` - - `ALTER ADD CONSTRAINT` - - `ALTER DROP CONSTRAINT` - - `ALTER TTL` - - `ALTER MATERIALIZE TTL` - - `ALTER SETTINGS` - - `ALTER MOVE PARTITION` - - `ALTER FETCH PARTITION` - - `ALTER FREEZE PARTITION` - - `ALTER VIEW` - - `ALTER VIEW REFRESH` - - `ALTER VIEW MODIFY QUERY` -- [CREATE](#grant-create) - - `CREATE DATABASE` - - `CREATE TABLE` - - `CREATE VIEW` - - `CREATE DICTIONARY` - - `CREATE TEMPORARY TABLE` -- [DROP](#grant-drop) - - `DROP DATABASE` - - `DROP TABLE` - - `DROP VIEW` - - `DROP DICTIONARY` -- [TRUNCATE](#grant-truncate) -- [OPTIMIZE](#grant-optimize) -- [SHOW](#grant-show) - - `SHOW DATABASES` - - `SHOW TABLES` - - `SHOW COLUMNS` - - `SHOW DICTIONARIES` -- [KILL QUERY](#grant-kill-query) -- [ACCESS MANAGEMENT](#grant-access-management) - - `CREATE USER` - - `ALTER USER` - - `DROP USER` - - `CREATE ROLE` - - `ALTER ROLE` - - `DROP ROLE` - - `CREATE ROW POLICY` - - `ALTER ROW POLICY` - - `DROP ROW POLICY` - - `CREATE QUOTA` - - `ALTER QUOTA` - - `DROP QUOTA` - - `CREATE SETTINGS PROFILE` - - `ALTER SETTINGS PROFILE` - - `DROP SETTINGS PROFILE` - - `SHOW ACCESS` - - `SHOW_USERS` - - `SHOW_ROLES` - - `SHOW_ROW_POLICIES` - - `SHOW_QUOTAS` - - `SHOW_SETTINGS_PROFILES` - - `ROLE ADMIN` -- [SYSTEM](#grant-system) - - `SYSTEM SHUTDOWN` - - `SYSTEM DROP CACHE` - - `SYSTEM DROP DNS CACHE` - - `SYSTEM DROP MARK CACHE` - - `SYSTEM DROP UNCOMPRESSED CACHE` - - `SYSTEM RELOAD` - - `SYSTEM RELOAD CONFIG` - - `SYSTEM RELOAD DICTIONARY` - - `SYSTEM RELOAD EMBEDDED DICTIONARIES` - - `SYSTEM MERGES` - - `SYSTEM TTL MERGES` - - `SYSTEM FETCHES` - - `SYSTEM MOVES` - - `SYSTEM SENDS` - - `SYSTEM DISTRIBUTED SENDS` - - `SYSTEM REPLICATED SENDS` - - `SYSTEM REPLICATION QUEUES` - - `SYSTEM SYNC REPLICA` - - `SYSTEM RESTART REPLICA` - - `SYSTEM FLUSH` - - `SYSTEM FLUSH DISTRIBUTED` - - `SYSTEM FLUSH LOGS` -- [INTROSPECTION](#grant-introspection) - - `addressToLine` - - `addressToSymbol` - - `demangle` -- [SOURCES](#grant-sources) - - `FILE` - - `URL` - - `REMOTE` - - `YSQL` - - `ODBC` - - `JDBC` - - `HDFS` - - `S3` -- [dictGet](#grant-dictget) - -Exemples de la façon dont cette hiérarchie est traitée: - -- Le `ALTER` privilège comprend tous les autres `ALTER*` privilège. -- `ALTER CONSTRAINT` comprendre `ALTER ADD CONSTRAINT` et `ALTER DROP CONSTRAINT` privilège. - -Les privilèges sont appliqués à différents niveaux. Connaissant un niveau suggère la syntaxe disponible pour le privilège. - -Les niveaux (du plus faible au plus élevé): - -- `COLUMN` — Privilege can be granted for column, table, database, or globally. -- `TABLE` — Privilege can be granted for table, database, or globally. -- `VIEW` — Privilege can be granted for view, database, or globally. -- `DICTIONARY` — Privilege can be granted for dictionary, database, or globally. -- `DATABASE` — Privilege can be granted for database or globally. -- `GLOBAL` — Privilege can be granted only globally. -- `GROUP` — Groups privileges of different levels. When `GROUP`- le privilège de niveau est accordé, seuls les privilèges du groupe sont accordés qui correspondent à la syntaxe utilisée. - -Exemples de syntaxe: - -- `GRANT SELECT(x) ON db.table TO user` -- `GRANT SELECT ON db.* TO user` - -Exemples de syntaxe refusée: - -- `GRANT CREATE USER(x) ON db.table TO user` -- `GRANT CREATE USER ON db.* TO user` - -Le privilège spécial [ALL](#grant-all) accorde tous les privilèges à un compte d'utilisateur ou à un rôle. - -Par défaut, un compte d'utilisateur ou un rôle a pas de privilèges. - -Si un utilisateur ou un rôle ont pas de privilèges qu'il s'affiche comme [NONE](#grant-none) privilège. - -Certaines requêtes par leur implémentation nécessitent un ensemble de privilèges. Par exemple, pour effectuer la [RENAME](misc.md#misc_operations-rename) requête vous avez besoin des privilèges suivants: `SELECT`, `CREATE TABLE`, `INSERT` et `DROP TABLE`. - -### SELECT {#grant-select} - -Permet d'effectuer des [SELECT](select/index.md) requête. - -Le niveau de privilège: `COLUMN`. - -**Description** - -L'utilisateur accordé avec ce privilège peut effectuer `SELECT` requêtes sur une liste spécifiée de colonnes dans la table et la base de données spécifiées. Si l'utilisateur inclut d'autres colonnes, une requête ne renvoie aucune donnée. - -Considérez le privilège suivant: - -``` sql -GRANT SELECT(x,y) ON db.table TO john -``` - -Ce privilège permet à `john` pour effectuer toute `SELECT` requête qui implique des données du `x` et/ou `y` les colonnes en `db.table`. Exemple, `SELECT x FROM db.table`. `john` ne pouvez pas effectuer de `SELECT z FROM db.table`. Le `SELECT * FROM db.table` aussi n'est pas disponible. En traitant cette requête, ClickHouse ne renvoie aucune donnée, même `x` et `y`. La seule exception est si une table contient uniquement `x` et `y` colonnes, dans ce cas ClickHouse renvoie toutes les données. - -### INSERT {#grant-insert} - -Permet d'effectuer des [INSERT](insert-into.md) requête. - -Le niveau de privilège: `COLUMN`. - -**Description** - -L'utilisateur accordé avec ce privilège peut effectuer `INSERT` requêtes sur une liste spécifiée de colonnes dans la table et la base de données spécifiées. Si l'utilisateur inclut d'autres colonnes, une requête n'insère aucune donnée. - -**Exemple** - -``` sql -GRANT INSERT(x,y) ON db.table TO john -``` - -Le privilège accordé permet `john` pour insérer des données à l' `x` et/ou `y` les colonnes en `db.table`. - -### ALTER {#grant-alter} - -Permet d'effectuer des [ALTER](alter.md) requêtes correspondant à la hiérarchie de privilèges suivante: - -- `ALTER`. Niveau: `COLUMN`. - - `ALTER TABLE`. Niveau: `GROUP` - - `ALTER UPDATE`. Niveau: `COLUMN`. Alias: `UPDATE` - - `ALTER DELETE`. Niveau: `COLUMN`. Alias: `DELETE` - - `ALTER COLUMN`. Niveau: `GROUP` - - `ALTER ADD COLUMN`. Niveau: `COLUMN`. Alias: `ADD COLUMN` - - `ALTER DROP COLUMN`. Niveau: `COLUMN`. Alias: `DROP COLUMN` - - `ALTER MODIFY COLUMN`. Niveau: `COLUMN`. Alias: `MODIFY COLUMN` - - `ALTER COMMENT COLUMN`. Niveau: `COLUMN`. Alias: `COMMENT COLUMN` - - `ALTER CLEAR COLUMN`. Niveau: `COLUMN`. Alias: `CLEAR COLUMN` - - `ALTER RENAME COLUMN`. Niveau: `COLUMN`. Alias: `RENAME COLUMN` - - `ALTER INDEX`. Niveau: `GROUP`. Alias: `INDEX` - - `ALTER ORDER BY`. Niveau: `TABLE`. Alias: `ALTER MODIFY ORDER BY`, `MODIFY ORDER BY` - - `ALTER ADD INDEX`. Niveau: `TABLE`. Alias: `ADD INDEX` - - `ALTER DROP INDEX`. Niveau: `TABLE`. Alias: `DROP INDEX` - - `ALTER MATERIALIZE INDEX`. Niveau: `TABLE`. Alias: `MATERIALIZE INDEX` - - `ALTER CLEAR INDEX`. Niveau: `TABLE`. Alias: `CLEAR INDEX` - - `ALTER CONSTRAINT`. Niveau: `GROUP`. Alias: `CONSTRAINT` - - `ALTER ADD CONSTRAINT`. Niveau: `TABLE`. Alias: `ADD CONSTRAINT` - - `ALTER DROP CONSTRAINT`. Niveau: `TABLE`. Alias: `DROP CONSTRAINT` - - `ALTER TTL`. Niveau: `TABLE`. Alias: `ALTER MODIFY TTL`, `MODIFY TTL` - - `ALTER MATERIALIZE TTL`. Niveau: `TABLE`. Alias: `MATERIALIZE TTL` - - `ALTER SETTINGS`. Niveau: `TABLE`. Alias: `ALTER SETTING`, `ALTER MODIFY SETTING`, `MODIFY SETTING` - - `ALTER MOVE PARTITION`. Niveau: `TABLE`. Alias: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART` - - `ALTER FETCH PARTITION`. Niveau: `TABLE`. Alias: `FETCH PARTITION` - - `ALTER FREEZE PARTITION`. Niveau: `TABLE`. Alias: `FREEZE PARTITION` - - `ALTER VIEW` Niveau: `GROUP` - - `ALTER VIEW REFRESH`. Niveau: `VIEW`. Alias: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW` - - `ALTER VIEW MODIFY QUERY`. Niveau: `VIEW`. Alias: `ALTER TABLE MODIFY QUERY` - -Exemples de la façon dont cette hiérarchie est traitée: - -- Le `ALTER` privilège comprend tous les autres `ALTER*` privilège. -- `ALTER CONSTRAINT` comprendre `ALTER ADD CONSTRAINT` et `ALTER DROP CONSTRAINT` privilège. - -**Note** - -- Le `MODIFY SETTING` privilège permet de modifier les paramètres du moteur de table. In n'affecte pas les paramètres ou les paramètres de configuration du serveur. -- Le `ATTACH` opération a besoin de la [CREATE](#grant-create) privilège. -- Le `DETACH` opération a besoin de la [DROP](#grant-drop) privilège. -- Pour arrêter la mutation par le [KILL MUTATION](misc.md#kill-mutation) requête, vous devez avoir un privilège pour commencer cette mutation. Par exemple, si vous voulez arrêter l' `ALTER UPDATE` requête, vous avez besoin du `ALTER UPDATE`, `ALTER TABLE`, ou `ALTER` privilège. - -### CREATE {#grant-create} - -Permet d'effectuer des [CREATE](create.md) et [ATTACH](misc.md#attach) DDL-requêtes correspondant à la hiérarchie de privilèges suivante: - -- `CREATE`. Niveau: `GROUP` - - `CREATE DATABASE`. Niveau: `DATABASE` - - `CREATE TABLE`. Niveau: `TABLE` - - `CREATE VIEW`. Niveau: `VIEW` - - `CREATE DICTIONARY`. Niveau: `DICTIONARY` - - `CREATE TEMPORARY TABLE`. Niveau: `GLOBAL` - -**Note** - -- Pour supprimer la table créée, l'utilisateur doit [DROP](#grant-drop). - -### DROP {#grant-drop} - -Permet d'effectuer des [DROP](misc.md#drop) et [DETACH](misc.md#detach) requêtes correspondant à la hiérarchie de privilèges suivante: - -- `DROP`. Niveau: - - `DROP DATABASE`. Niveau: `DATABASE` - - `DROP TABLE`. Niveau: `TABLE` - - `DROP VIEW`. Niveau: `VIEW` - - `DROP DICTIONARY`. Niveau: `DICTIONARY` - -### TRUNCATE {#grant-truncate} - -Permet d'effectuer des [TRUNCATE](misc.md#truncate-statement) requête. - -Le niveau de privilège: `TABLE`. - -### OPTIMIZE {#grant-optimize} - -Permet d'effectuer les [OPTIMIZE TABLE](misc.md#misc_operations-optimize) requête. - -Le niveau de privilège: `TABLE`. - -### SHOW {#grant-show} - -Permet d'effectuer des `SHOW`, `DESCRIBE`, `USE`, et `EXISTS` requêtes, correspondant à la hiérarchie suivante des privilèges: - -- `SHOW`. Niveau: `GROUP` - - `SHOW DATABASES`. Niveau: `DATABASE`. Permet d'exécuter des `SHOW DATABASES`, `SHOW CREATE DATABASE`, `USE ` requête. - - `SHOW TABLES`. Niveau: `TABLE`. Permet d'exécuter des `SHOW TABLES`, `EXISTS `, `CHECK
` requête. - - `SHOW COLUMNS`. Niveau: `COLUMN`. Permet d'exécuter des `SHOW CREATE TABLE`, `DESCRIBE` requête. - - `SHOW DICTIONARIES`. Niveau: `DICTIONARY`. Permet d'exécuter des `SHOW DICTIONARIES`, `SHOW CREATE DICTIONARY`, `EXISTS ` requête. - -**Note** - -Un utilisateur a le `SHOW` privilège s'il a un autre privilège concernant la table, le dictionnaire ou la base de données spécifiés. - -### KILL QUERY {#grant-kill-query} - -Permet d'effectuer les [KILL](misc.md#kill-query-statement) requêtes correspondant à la hiérarchie de privilèges suivante: - -Le niveau de privilège: `GLOBAL`. - -**Note** - -`KILL QUERY` privilège permet à un utilisateur de tuer les requêtes des autres utilisateurs. - -### ACCESS MANAGEMENT {#grant-access-management} - -Permet à un utilisateur d'effectuer des requêtes qui gèrent les utilisateurs, les rôles et les stratégies de ligne. - -- `ACCESS MANAGEMENT`. Niveau: `GROUP` - - `CREATE USER`. Niveau: `GLOBAL` - - `ALTER USER`. Niveau: `GLOBAL` - - `DROP USER`. Niveau: `GLOBAL` - - `CREATE ROLE`. Niveau: `GLOBAL` - - `ALTER ROLE`. Niveau: `GLOBAL` - - `DROP ROLE`. Niveau: `GLOBAL` - - `ROLE ADMIN`. Niveau: `GLOBAL` - - `CREATE ROW POLICY`. Niveau: `GLOBAL`. Alias: `CREATE POLICY` - - `ALTER ROW POLICY`. Niveau: `GLOBAL`. Alias: `ALTER POLICY` - - `DROP ROW POLICY`. Niveau: `GLOBAL`. Alias: `DROP POLICY` - - `CREATE QUOTA`. Niveau: `GLOBAL` - - `ALTER QUOTA`. Niveau: `GLOBAL` - - `DROP QUOTA`. Niveau: `GLOBAL` - - `CREATE SETTINGS PROFILE`. Niveau: `GLOBAL`. Alias: `CREATE PROFILE` - - `ALTER SETTINGS PROFILE`. Niveau: `GLOBAL`. Alias: `ALTER PROFILE` - - `DROP SETTINGS PROFILE`. Niveau: `GLOBAL`. Alias: `DROP PROFILE` - - `SHOW ACCESS`. Niveau: `GROUP` - - `SHOW_USERS`. Niveau: `GLOBAL`. Alias: `SHOW CREATE USER` - - `SHOW_ROLES`. Niveau: `GLOBAL`. Alias: `SHOW CREATE ROLE` - - `SHOW_ROW_POLICIES`. Niveau: `GLOBAL`. Alias: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY` - - `SHOW_QUOTAS`. Niveau: `GLOBAL`. Alias: `SHOW CREATE QUOTA` - - `SHOW_SETTINGS_PROFILES`. Niveau: `GLOBAL`. Alias: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE` - -Le `ROLE ADMIN` le privilège permet à un utilisateur d'accorder et de révoquer tous les rôles, y compris ceux qui ne lui sont pas accordés avec l'option admin. - -### SYSTEM {#grant-system} - -Permet à un utilisateur d'effectuer la [SYSTEM](system.md) requêtes correspondant à la hiérarchie de privilèges suivante. - -- `SYSTEM`. Niveau: `GROUP` - - `SYSTEM SHUTDOWN`. Niveau: `GLOBAL`. Alias: `SYSTEM KILL`, `SHUTDOWN` - - `SYSTEM DROP CACHE`. Alias: `DROP CACHE` - - `SYSTEM DROP DNS CACHE`. Niveau: `GLOBAL`. Alias: `SYSTEM DROP DNS`, `DROP DNS CACHE`, `DROP DNS` - - `SYSTEM DROP MARK CACHE`. Niveau: `GLOBAL`. Alias: `SYSTEM DROP MARK`, `DROP MARK CACHE`, `DROP MARKS` - - `SYSTEM DROP UNCOMPRESSED CACHE`. Niveau: `GLOBAL`. Alias: `SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, `DROP UNCOMPRESSED` - - `SYSTEM RELOAD`. Niveau: `GROUP` - - `SYSTEM RELOAD CONFIG`. Niveau: `GLOBAL`. Alias: `RELOAD CONFIG` - - `SYSTEM RELOAD DICTIONARY`. Niveau: `GLOBAL`. Alias: `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARY`, `RELOAD DICTIONARIES` - - `SYSTEM RELOAD EMBEDDED DICTIONARIES`. Niveau: `GLOBAL`. Alias: R`ELOAD EMBEDDED DICTIONARIES` - - `SYSTEM MERGES`. Niveau: `TABLE`. Alias: `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, `START MERGES` - - `SYSTEM TTL MERGES`. Niveau: `TABLE`. Alias: `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, `START TTL MERGES` - - `SYSTEM FETCHES`. Niveau: `TABLE`. Alias: `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, `START FETCHES` - - `SYSTEM MOVES`. Niveau: `TABLE`. Alias: `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, `START MOVES` - - `SYSTEM SENDS`. Niveau: `GROUP`. Alias: `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, `START SENDS` - - `SYSTEM DISTRIBUTED SENDS`. Niveau: `TABLE`. Alias: `SYSTEM STOP DISTRIBUTED SENDS`, `SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, `START DISTRIBUTED SENDS` - - `SYSTEM REPLICATED SENDS`. Niveau: `TABLE`. Alias: `SYSTEM STOP REPLICATED SENDS`, `SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, `START REPLICATED SENDS` - - `SYSTEM REPLICATION QUEUES`. Niveau: `TABLE`. Alias: `SYSTEM STOP REPLICATION QUEUES`, `SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, `START REPLICATION QUEUES` - - `SYSTEM SYNC REPLICA`. Niveau: `TABLE`. Alias: `SYNC REPLICA` - - `SYSTEM RESTART REPLICA`. Niveau: `TABLE`. Alias: `RESTART REPLICA` - - `SYSTEM FLUSH`. Niveau: `GROUP` - - `SYSTEM FLUSH DISTRIBUTED`. Niveau: `TABLE`. Alias: `FLUSH DISTRIBUTED` - - `SYSTEM FLUSH LOGS`. Niveau: `GLOBAL`. Alias: `FLUSH LOGS` - -Le `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilège implicitement accordé par le `SYSTEM RELOAD DICTIONARY ON *.*` privilège. - -### INTROSPECTION {#grant-introspection} - -Permet l'utilisation de [introspection](../../operations/optimizing-performance/sampling-query-profiler.md) fonction. - -- `INTROSPECTION`. Niveau: `GROUP`. Alias: `INTROSPECTION FUNCTIONS` - - `addressToLine`. Niveau: `GLOBAL` - - `addressToSymbol`. Niveau: `GLOBAL` - - `demangle`. Niveau: `GLOBAL` - -### SOURCES {#grant-sources} - -Permet d'utiliser des sources de données externes. S'applique à [moteurs de table](../../engines/table-engines/index.md) et [les fonctions de table](../table-functions/index.md#table-functions). - -- `SOURCES`. Niveau: `GROUP` - - `FILE`. Niveau: `GLOBAL` - - `URL`. Niveau: `GLOBAL` - - `REMOTE`. Niveau: `GLOBAL` - - `YSQL`. Niveau: `GLOBAL` - - `ODBC`. Niveau: `GLOBAL` - - `JDBC`. Niveau: `GLOBAL` - - `HDFS`. Niveau: `GLOBAL` - - `S3`. Niveau: `GLOBAL` - -Le `SOURCES` privilège permet l'utilisation de toutes les sources. Vous pouvez également accorder un privilège pour chaque source individuellement. Pour utiliser les sources, vous avez besoin de privilèges supplémentaires. - -Exemple: - -- Pour créer une table avec [Moteur de table MySQL](../../engines/table-engines/integrations/mysql.md), vous avez besoin `CREATE TABLE (ON db.table_name)` et `MYSQL` privilège. -- L'utilisation de la [fonction de table mysql](../table-functions/mysql.md), vous avez besoin `CREATE TEMPORARY TABLE` et `MYSQL` privilège. - -### dictGet {#grant-dictget} - -- `dictGet`. Alias: `dictHas`, `dictGetHierarchy`, `dictIsIn` - -Permet à un utilisateur d'exécuter [dictGet](../functions/ext-dict-functions.md#dictget), [dictHas](../functions/ext-dict-functions.md#dicthas), [dictGetHierarchy](../functions/ext-dict-functions.md#dictgethierarchy), [dictisine](../functions/ext-dict-functions.md#dictisin) fonction. - -Niveau de privilège: `DICTIONARY`. - -**Exemple** - -- `GRANT dictGet ON mydb.mydictionary TO john` -- `GRANT dictGet ON mydictionary TO john` - -### ALL {#grant-all} - -Les subventions de tous les privilèges sur l'entité réglementée à un compte d'utilisateur ou un rôle. - -### NONE {#grant-none} - -N'accorde pas de privilèges. - -### ADMIN OPTION {#admin-option-privilege} - -Le `ADMIN OPTION` le privilège permet à un utilisateur d'accorder son rôle à un autre utilisateur. - -[Article Original](https://clickhouse.tech/docs/en/query_language/grant/) diff --git a/docs/fr/sql-reference/statements/index.md b/docs/fr/sql-reference/statements/index.md deleted file mode 100644 index f08d64cee39..00000000000 --- a/docs/fr/sql-reference/statements/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "D\xE9claration" -toc_priority: 31 ---- - - diff --git a/docs/fr/sql-reference/statements/insert-into.md b/docs/fr/sql-reference/statements/insert-into.md deleted file mode 100644 index 987594bae65..00000000000 --- a/docs/fr/sql-reference/statements/insert-into.md +++ /dev/null @@ -1,80 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 34 -toc_title: INSERT INTO ---- - -## INSERT {#insert} - -L'ajout de données. - -Format de requête de base: - -``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... -``` - -La requête peut spécifier une liste de colonnes à insérer `[(c1, c2, c3)]`. Dans ce cas, le reste des colonnes sont remplis avec: - -- Les valeurs calculées à partir `DEFAULT` expressions spécifiées dans la définition de la table. -- Zéros et chaînes vides, si `DEFAULT` les expressions ne sont pas définies. - -Si [strict_insert_defaults=1](../../operations/settings/settings.md), les colonnes qui n'ont pas `DEFAULT` défini doit être répertorié dans la requête. - -Les données peuvent être transmises à L'INSERT dans n'importe quel [format](../../interfaces/formats.md#formats) soutenu par ClickHouse. Le format doit être spécifié explicitement dans la requête: - -``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set -``` - -For example, the following query format is identical to the basic version of INSERT … VALUES: - -``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... -``` - -ClickHouse supprime tous les espaces et un saut de ligne (s'il y en a un) avant les données. Lors de la formation d'une requête, nous recommandons de placer les données sur une nouvelle ligne après les opérateurs de requête (ceci est important si les données commencent par des espaces). - -Exemple: - -``` sql -INSERT INTO t FORMAT TabSeparated -11 Hello, world! -22 Qwerty -``` - -Vous pouvez insérer des données séparément de la requête à l'aide du client de ligne de commande ou de L'interface HTTP. Pour plus d'informations, consultez la section “[Interface](../../interfaces/index.md#interfaces)”. - -### Contraintes {#constraints} - -Si la table a [contraintes](create.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. - -### Insertion des résultats de `SELECT` {#insert_query_insert-select} - -``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... -``` - -Les colonnes sont mappées en fonction de leur position dans la clause SELECT. Cependant, leurs noms dans L'expression SELECT et la table pour INSERT peuvent différer. Si nécessaire, la coulée de type est effectuée. - -Aucun des formats de données à l'exception des Valeurs permettent de définir des valeurs d'expressions telles que `now()`, `1 + 2` et ainsi de suite. Le format des valeurs permet une utilisation limitée des expressions, mais ce n'est pas recommandé, car dans ce cas, un code inefficace est utilisé pour leur exécution. - -Les autres requêtes de modification des parties de données ne sont pas prises en charge: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`. -Cependant, vous pouvez supprimer les anciennes données en utilisant `ALTER TABLE ... DROP PARTITION`. - -`FORMAT` la clause doit être spécifié à la fin de la requête si `SELECT` la clause contient la fonction de table [entrée()](../table-functions/input.md). - -### Considérations De Performance {#performance-considerations} - -`INSERT` trie les données d'entrée par la clé primaire et les divise en partitions par une clé de partition. Si vous insérez des données dans plusieurs partitions à la fois, cela peut réduire considérablement les performances de l' `INSERT` requête. Pour éviter cela: - -- Ajoutez des données en lots assez importants, tels que 100 000 lignes à la fois. -- Groupez les données par une clé de partition avant de les télécharger sur ClickHouse. - -Les performances ne diminueront pas si: - -- Les données sont ajoutées en temps réel. -- Vous téléchargez des données qui sont généralement triées par heure. - -[Article Original](https://clickhouse.tech/docs/en/query_language/insert_into/) diff --git a/docs/fr/sql-reference/statements/misc.md b/docs/fr/sql-reference/statements/misc.md deleted file mode 100644 index 4631f856266..00000000000 --- a/docs/fr/sql-reference/statements/misc.md +++ /dev/null @@ -1,358 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: Autre ---- - -# Diverses Requêtes {#miscellaneous-queries} - -## ATTACH {#attach} - -Cette requête est exactement la même que `CREATE`, mais - -- Au lieu de la parole `CREATE` il utilise le mot `ATTACH`. -- La requête ne crée pas de données sur le disque, mais suppose que les données sont déjà aux endroits appropriés, et ajoute simplement des informations sur la table au serveur. - Après avoir exécuté une requête ATTACH, le serveur connaîtra l'existence de la table. - -Si la table a été précédemment détachée (`DETACH`), ce qui signifie que sa structure est connue, vous pouvez utiliser un raccourci sans définir la structure. - -``` sql -ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Cette requête est utilisée lors du démarrage du serveur. Le serveur stocke les métadonnées de la table sous forme de fichiers avec `ATTACH` requêtes, qu'il exécute simplement au lancement (à l'exception des tables système, qui sont explicitement créées sur le serveur). - -## CHECK TABLE {#check-table} - -Vérifie si les données de la table sont corrompues. - -``` sql -CHECK TABLE [db.]name -``` - -Le `CHECK TABLE` requête compare réelle des tailles de fichier avec les valeurs attendues qui sont stockés sur le serveur. Si le fichier tailles ne correspondent pas aux valeurs stockées, cela signifie que les données sont endommagées. Cela peut être causé, par exemple, par un plantage du système lors de l'exécution de la requête. - -La réponse de la requête contient `result` colonne avec une seule ligne. La ligne a une valeur de -[Booléen](../../sql-reference/data-types/boolean.md) type: - -- 0 - les données de la table sont corrompues. -- 1 - les données maintiennent l'intégrité. - -Le `CHECK TABLE` query prend en charge les moteurs de table suivants: - -- [Journal](../../engines/table-engines/log-family/log.md) -- [TinyLog](../../engines/table-engines/log-family/tinylog.md) -- [StripeLog](../../engines/table-engines/log-family/stripelog.md) -- [Famille MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) - -Effectué sur les tables avec un autre moteur de table provoque une exception. - -Les moteurs de la `*Log` la famille ne fournit pas de récupération automatique des données en cas d'échec. L'utilisation de la `CHECK TABLE` requête pour suivre la perte de données en temps opportun. - -Pour `MergeTree` moteurs de la famille, le `CHECK TABLE` query affiche un État de vérification pour chaque partie de données individuelle d'une table sur le serveur local. - -**Si les données sont corrompues** - -Si la table est corrompue, vous pouvez copier les données non corrompues dans une autre table. Pour ce faire: - -1. Créez une nouvelle table avec la même structure que la table endommagée. Pour ce faire exécutez la requête `CREATE TABLE AS `. -2. Définir le [max_threads](../../operations/settings/settings.md#settings-max_threads) la valeur 1 pour traiter la requête suivante dans un seul thread. Pour ce faire, exécutez la requête `SET max_threads = 1`. -3. Exécuter la requête `INSERT INTO SELECT * FROM `. Cette demande copie les données non corrompues de la table endommagée vers une autre table. Seules les données avant la partie corrompue seront copiées. -4. Redémarrez l' `clickhouse-client` pour réinitialiser l' `max_threads` valeur. - -## DESCRIBE TABLE {#misc-describe-table} - -``` sql -DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] -``` - -Renvoie ce qui suit `String` les colonnes de type: - -- `name` — Column name. -- `type`— Column type. -- `default_type` — Clause that is used in [expression par défaut](create.md#create-default-values) (`DEFAULT`, `MATERIALIZED` ou `ALIAS`). Column contient une chaîne vide, si l'expression par défaut n'est pas spécifiée. -- `default_expression` — Value specified in the `DEFAULT` clause. -- `comment_expression` — Comment text. - -Les structures de données imbriquées sont sorties dans “expanded” format. Chaque colonne est affichée séparément, avec le nom après un point. - -## DETACH {#detach} - -Supprime les informations sur le ‘name’ table du serveur. Le serveur cesse de connaître l'existence de la table. - -``` sql -DETACH TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Cela ne supprime pas les données ou les métadonnées de la table. Lors du prochain lancement du serveur, le serveur Lira les métadonnées et découvrira à nouveau la table. -De même, un “detached” tableau peut être re-attaché en utilisant le `ATTACH` requête (à l'exception des tables système, qui n'ont pas de stocker les métadonnées pour eux). - -Il n'y a pas de `DETACH DATABASE` requête. - -## DROP {#drop} - -Cette requête a deux types: `DROP DATABASE` et `DROP TABLE`. - -``` sql -DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] -``` - -Supprime toutes les tables à l'intérieur de la ‘db’ la base de données, puis supprime le ‘db’ la base de données elle-même. -Si `IF EXISTS` est spécifié, il ne renvoie pas d'erreur si la base de données n'existe pas. - -``` sql -DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Supprime la table. -Si `IF EXISTS` est spécifié, il ne renvoie pas d'erreur si la table n'existe pas ou si la base de données n'existe pas. - - DROP DICTIONARY [IF EXISTS] [db.]name - -Delets le dictionnaire. -Si `IF EXISTS` est spécifié, il ne renvoie pas d'erreur si la table n'existe pas ou si la base de données n'existe pas. - -## DROP USER {#drop-user-statement} - -Supprime un utilisateur. - -### Syntaxe {#drop-user-syntax} - -``` sql -DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - -## DROP ROLE {#drop-role-statement} - -Supprime un rôle. - -Le rôle supprimé est révoqué de toutes les entités où il a été accordé. - -### Syntaxe {#drop-role-syntax} - -``` sql -DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - -## DROP ROW POLICY {#drop-row-policy-statement} - -Supprime une stratégie de ligne. - -La stratégie de ligne supprimée est révoquée de toutes les entités sur lesquelles elle a été affectée. - -### Syntaxe {#drop-row-policy-syntax} - -``` sql -DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] -``` - -## DROP QUOTA {#drop-quota-statement} - -Supprime un quota. - -Le quota supprimé est révoqué de toutes les entités où il a été affecté. - -### Syntaxe {#drop-quota-syntax} - -``` sql -DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - -## DROP SETTINGS PROFILE {#drop-settings-profile-statement} - -Supprime un quota. - -Le quota supprimé est révoqué de toutes les entités où il a été affecté. - -### Syntaxe {#drop-settings-profile-syntax} - -``` sql -DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] -``` - -## EXISTS {#exists-statement} - -``` sql -EXISTS [TEMPORARY] [TABLE|DICTIONARY] [db.]name [INTO OUTFILE filename] [FORMAT format] -``` - -Renvoie un seul `UInt8`- type colonne, qui contient la valeur unique `0` si la table ou base de données n'existe pas, ou `1` si la table existe dans la base de données spécifiée. - -## KILL QUERY {#kill-query-statement} - -``` sql -KILL QUERY [ON CLUSTER cluster] - WHERE - [SYNC|ASYNC|TEST] - [FORMAT format] -``` - -Tente de mettre fin de force aux requêtes en cours d'exécution. -Les requêtes à terminer sont sélectionnées dans le système.processus en utilisant les critères définis dans le `WHERE` la clause de la `KILL` requête. - -Exemple: - -``` sql --- Forcibly terminates all queries with the specified query_id: -KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' - --- Synchronously terminates all queries run by 'username': -KILL QUERY WHERE user='username' SYNC -``` - -Les utilisateurs en lecture seule peuvent uniquement arrêter leurs propres requêtes. - -Par défaut, la version asynchrone des requêtes est utilisé (`ASYNC`), qui n'attend pas la confirmation que les requêtes se sont arrêtées. - -La version synchrone (`SYNC`) attend que toutes les requêtes d'arrêter et affiche des informations sur chaque processus s'arrête. -La réponse contient l' `kill_status` la colonne, qui peut prendre les valeurs suivantes: - -1. ‘finished’ – The query was terminated successfully. -2. ‘waiting’ – Waiting for the query to end after sending it a signal to terminate. -3. The other values ​​explain why the query can't be stopped. - -Une requête de test (`TEST`) vérifie uniquement les droits de l'utilisateur et affiche une liste de requêtes à arrêter. - -## KILL MUTATION {#kill-mutation} - -``` sql -KILL MUTATION [ON CLUSTER cluster] - WHERE - [TEST] - [FORMAT format] -``` - -Essaie d'annuler et supprimer [mutation](alter.md#alter-mutations) actuellement en cours d'exécution. Les Mutations à annuler sont sélectionnées parmi [`system.mutations`](../../operations/system-tables.md#system_tables-mutations) tableau à l'aide du filtre spécifié par le `WHERE` la clause de la `KILL` requête. - -Une requête de test (`TEST`) vérifie uniquement les droits de l'utilisateur et affiche une liste de requêtes à arrêter. - -Exemple: - -``` sql --- Cancel and remove all mutations of the single table: -KILL MUTATION WHERE database = 'default' AND table = 'table' - --- Cancel the specific mutation: -KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' -``` - -The query is useful when a mutation is stuck and cannot finish (e.g. if some function in the mutation query throws an exception when applied to the data contained in the table). - -Les modifications déjà apportées par la mutation ne sont pas annulées. - -## OPTIMIZE {#misc_operations-optimize} - -``` sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] [DEDUPLICATE] -``` - -Cette requête tente d'initialiser une fusion non programmée de parties de données pour les tables avec un moteur de [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) famille. - -Le `OPTMIZE` la requête est également prise en charge pour [MaterializedView](../../engines/table-engines/special/materializedview.md) et la [Tampon](../../engines/table-engines/special/buffer.md) moteur. Les autres moteurs de table ne sont pas pris en charge. - -Lorsque `OPTIMIZE` est utilisé avec le [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) famille de moteurs de table, ClickHouse crée une tâche pour la fusion et attend l'exécution sur tous les nœuds (si le `replication_alter_partitions_sync` paramètre est activé). - -- Si `OPTIMIZE` n'effectue pas de fusion pour une raison quelconque, il ne notifie pas le client. Pour activer les notifications, utilisez [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) paramètre. -- Si vous spécifiez un `PARTITION`, seule la partition spécifiée est optimisé. [Comment définir l'expression de la partition](alter.md#alter-how-to-specify-part-expr). -- Si vous spécifiez `FINAL`, l'optimisation est effectuée, même lorsque toutes les données sont déjà dans une partie. -- Si vous spécifiez `DEDUPLICATE`, alors des lignes complètement identiques seront dédupliquées (toutes les colonnes sont comparées), cela n'a de sens que pour le moteur MergeTree. - -!!! warning "Avertissement" - `OPTIMIZE` ne peut pas réparer le “Too many parts” erreur. - -## RENAME {#misc_operations-rename} - -Renomme une ou plusieurs tables. - -``` sql -RENAME TABLE [db11.]name11 TO [db12.]name12, [db21.]name21 TO [db22.]name22, ... [ON CLUSTER cluster] -``` - -Toutes les tables sont renommées sous verrouillage global. Renommer des tables est une opération légère. Si vous avez indiqué une autre base de données après TO, la table sera déplacée vers cette base de données. Cependant, les répertoires contenant des bases de données doivent résider dans le même système de fichiers (sinon, une erreur est renvoyée). - -## SET {#query-set} - -``` sql -SET param = value -``` - -Assigner `value` à l' `param` [paramètre](../../operations/settings/index.md) pour la session en cours. Vous ne pouvez pas modifier [les paramètres du serveur](../../operations/server-configuration-parameters/index.md) de cette façon. - -Vous pouvez également définir toutes les valeurs de certains paramètres de profil dans une seule requête. - -``` sql -SET profile = 'profile-name-from-the-settings-file' -``` - -Pour plus d'informations, voir [Paramètre](../../operations/settings/settings.md). - -## SET ROLE {#set-role-statement} - -Active les rôles pour l'utilisateur actuel. - -### Syntaxe {#set-role-syntax} - -``` sql -SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]} -``` - -## SET DEFAULT ROLE {#set-default-role-statement} - -Définit les rôles par défaut à un utilisateur. - -Les rôles par défaut sont automatiquement activés lors de la connexion de l'utilisateur. Vous pouvez définir par défaut uniquement les rôles précédemment accordés. Si le rôle n'est pas accordé à un utilisateur, ClickHouse lève une exception. - -### Syntaxe {#set-default-role-syntax} - -``` sql -SET DEFAULT ROLE {NONE | role [,...] | ALL | ALL EXCEPT role [,...]} TO {user|CURRENT_USER} [,...] -``` - -### Exemple {#set-default-role-examples} - -Définir plusieurs rôles par défaut à un utilisateur: - -``` sql -SET DEFAULT ROLE role1, role2, ... TO user -``` - -Définissez tous les rôles accordés par défaut sur un utilisateur: - -``` sql -SET DEFAULT ROLE ALL TO user -``` - -Purger les rôles par défaut d'un utilisateur: - -``` sql -SET DEFAULT ROLE NONE TO user -``` - -Définissez tous les rôles accordés par défaut à l'exception de certains d'entre eux: - -``` sql -SET DEFAULT ROLE ALL EXCEPT role1, role2 TO user -``` - -## TRUNCATE {#truncate-statement} - -``` sql -TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] -``` - -Supprime toutes les données d'une table. Lorsque la clause `IF EXISTS` est omis, la requête renvoie une erreur si la table n'existe pas. - -Le `TRUNCATE` la requête n'est pas prise en charge pour [Vue](../../engines/table-engines/special/view.md), [Fichier](../../engines/table-engines/special/file.md), [URL](../../engines/table-engines/special/url.md) et [NULL](../../engines/table-engines/special/null.md) table des moteurs. - -## USE {#use} - -``` sql -USE db -``` - -Vous permet de définir la base de données actuelle pour la session. -La base de données actuelle est utilisée pour rechercher des tables si la base de données n'est pas explicitement définie dans la requête avec un point avant le nom de la table. -Cette requête ne peut pas être faite lors de l'utilisation du protocole HTTP, car il n'y a pas de concept de session. - -[Article Original](https://clickhouse.tech/docs/en/query_language/misc/) diff --git a/docs/fr/sql-reference/statements/revoke.md b/docs/fr/sql-reference/statements/revoke.md deleted file mode 100644 index 6137cc30f8c..00000000000 --- a/docs/fr/sql-reference/statements/revoke.md +++ /dev/null @@ -1,50 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: REVOKE ---- - -# REVOKE {#revoke} - -Révoque les privilèges des utilisateurs ou rôles. - -## Syntaxe {#revoke-syntax} - -**Révocation des privilèges des utilisateurs** - -``` sql -REVOKE [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...] -``` - -**Révocation des rôles des utilisateurs** - -``` sql -REVOKE [ON CLUSTER cluster_name] [ADMIN OPTION FOR] role [,...] FROM {user | role | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...] -``` - -## Description {#revoke-description} - -Pour révoquer certains privilèges, vous pouvez utiliser un privilège de portée plus large que vous envisagez de révoquer. Par exemple, si un utilisateur a la `SELECT (x,y)` privilège, administrateur peut effectuer `REVOKE SELECT(x,y) ...`, ou `REVOKE SELECT * ...` ou même `REVOKE ALL PRIVILEGES ...` requête de révoquer ce privilège. - -### Révocations Partielles {#partial-revokes-dscr} - -Vous pouvez révoquer une partie d'un privilège. Par exemple, si un utilisateur a la `SELECT *.*` Privilège vous pouvez révoquer un privilège pour lire les données d'une table ou d'une base de données. - -## Exemple {#revoke-example} - -Subvention de l' `john` compte utilisateur avec le privilège de sélectionner parmi toutes les bases de données `accounts` un: - -``` sql -GRANT SELECT ON *.* TO john; -REVOKE SELECT ON accounts.* FROM john; -``` - -Subvention de l' `mira` compte utilisateur avec le privilège de sélectionner parmi toutes les colonnes `accounts.staff` tableau à l'exception de la `wage` un. - -``` sql -GRANT SELECT ON accounts.staff TO mira; -REVOKE SELECT(wage) ON accounts.staff FROM mira; -``` - -{## [Article Original](https://clickhouse.tech/docs/en/operations/settings/settings/) ##} diff --git a/docs/fr/sql-reference/statements/select/array-join.md b/docs/fr/sql-reference/statements/select/array-join.md deleted file mode 100644 index 07b27d5d16c..00000000000 --- a/docs/fr/sql-reference/statements/select/array-join.md +++ /dev/null @@ -1,282 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause de jointure de tableau {#select-array-join-clause} - -C'est une opération courante pour les tables qui contiennent une colonne de tableau pour produire une nouvelle table qui a une colonne avec chaque élément de tableau individuel de cette colonne initiale, tandis que les valeurs des autres colonnes sont dupliquées. C'est le cas de fond de ce `ARRAY JOIN` la clause le fait. - -Son nom vient du fait qu'il peut être regardé comme l'exécution de `JOIN` avec un tableau ou une structure de données imbriquée. L'intention est similaire à la [arrayJoin](../../functions/array-join.md#functions_arrayjoin) fonction, mais la fonctionnalité de la clause est plus large. - -Syntaxe: - -``` sql -SELECT -FROM -[LEFT] ARRAY JOIN -[WHERE|PREWHERE ] -... -``` - -Vous ne pouvez en spécifier qu'un `ARRAY JOIN` la clause dans un `SELECT` requête. - -Types pris en charge de `ARRAY JOIN` sont énumérés ci-dessous: - -- `ARRAY JOIN` - Dans le cas de base, les tableaux vides ne sont pas inclus dans le résultat de `JOIN`. -- `LEFT ARRAY JOIN` - Le résultat de `JOIN` contient des lignes avec des tableaux vides. La valeur d'un tableau vide est définie sur la valeur par défaut pour le type d'élément de tableau (généralement 0, chaîne vide ou NULL). - -## Exemples de jointure de tableau de base {#basic-array-join-examples} - -Les exemples ci-dessous illustrent l'utilisation de la `ARRAY JOIN` et `LEFT ARRAY JOIN` clause. Créons une table avec un [Tableau](../../../sql-reference/data-types/array.md) tapez colonne et insérez des valeurs dedans: - -``` sql -CREATE TABLE arrays_test -( - s String, - arr Array(UInt8) -) ENGINE = Memory; - -INSERT INTO arrays_test -VALUES ('Hello', [1,2]), ('World', [3,4,5]), ('Goodbye', []); -``` - -``` text -┌─s───────────┬─arr─────┐ -│ Hello │ [1,2] │ -│ World │ [3,4,5] │ -│ Goodbye │ [] │ -└─────────────┴─────────┘ -``` - -L'exemple ci-dessous utilise la `ARRAY JOIN` clause: - -``` sql -SELECT s, arr -FROM arrays_test -ARRAY JOIN arr; -``` - -``` text -┌─s─────┬─arr─┐ -│ Hello │ 1 │ -│ Hello │ 2 │ -│ World │ 3 │ -│ World │ 4 │ -│ World │ 5 │ -└───────┴─────┘ -``` - -L'exemple suivant utilise l' `LEFT ARRAY JOIN` clause: - -``` sql -SELECT s, arr -FROM arrays_test -LEFT ARRAY JOIN arr; -``` - -``` text -┌─s───────────┬─arr─┐ -│ Hello │ 1 │ -│ Hello │ 2 │ -│ World │ 3 │ -│ World │ 4 │ -│ World │ 5 │ -│ Goodbye │ 0 │ -└─────────────┴─────┘ -``` - -## À L'Aide D'Alias {#using-aliases} - -Un alias peut être spécifié pour un tableau `ARRAY JOIN` clause. Dans ce cas, un élément de tableau peut être consulté par ce pseudonyme, mais le tableau lui-même est accessible par le nom d'origine. Exemple: - -``` sql -SELECT s, arr, a -FROM arrays_test -ARRAY JOIN arr AS a; -``` - -``` text -┌─s─────┬─arr─────┬─a─┐ -│ Hello │ [1,2] │ 1 │ -│ Hello │ [1,2] │ 2 │ -│ World │ [3,4,5] │ 3 │ -│ World │ [3,4,5] │ 4 │ -│ World │ [3,4,5] │ 5 │ -└───────┴─────────┴───┘ -``` - -En utilisant des alias, vous pouvez effectuer `ARRAY JOIN` avec un groupe externe. Exemple: - -``` sql -SELECT s, arr_external -FROM arrays_test -ARRAY JOIN [1, 2, 3] AS arr_external; -``` - -``` text -┌─s───────────┬─arr_external─┐ -│ Hello │ 1 │ -│ Hello │ 2 │ -│ Hello │ 3 │ -│ World │ 1 │ -│ World │ 2 │ -│ World │ 3 │ -│ Goodbye │ 1 │ -│ Goodbye │ 2 │ -│ Goodbye │ 3 │ -└─────────────┴──────────────┘ -``` - -Plusieurs tableaux peuvent être séparés par des virgules `ARRAY JOIN` clause. Dans ce cas, `JOIN` est effectuée avec eux simultanément (la somme directe, pas le produit cartésien). Notez que tous les tableaux doivent avoir la même taille. Exemple: - -``` sql -SELECT s, arr, a, num, mapped -FROM arrays_test -ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped; -``` - -``` text -┌─s─────┬─arr─────┬─a─┬─num─┬─mapped─┐ -│ Hello │ [1,2] │ 1 │ 1 │ 2 │ -│ Hello │ [1,2] │ 2 │ 2 │ 3 │ -│ World │ [3,4,5] │ 3 │ 1 │ 4 │ -│ World │ [3,4,5] │ 4 │ 2 │ 5 │ -│ World │ [3,4,5] │ 5 │ 3 │ 6 │ -└───────┴─────────┴───┴─────┴────────┘ -``` - -L'exemple ci-dessous utilise la [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) fonction: - -``` sql -SELECT s, arr, a, num, arrayEnumerate(arr) -FROM arrays_test -ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num; -``` - -``` text -┌─s─────┬─arr─────┬─a─┬─num─┬─arrayEnumerate(arr)─┐ -│ Hello │ [1,2] │ 1 │ 1 │ [1,2] │ -│ Hello │ [1,2] │ 2 │ 2 │ [1,2] │ -│ World │ [3,4,5] │ 3 │ 1 │ [1,2,3] │ -│ World │ [3,4,5] │ 4 │ 2 │ [1,2,3] │ -│ World │ [3,4,5] │ 5 │ 3 │ [1,2,3] │ -└───────┴─────────┴───┴─────┴─────────────────────┘ -``` - -## Jointure de tableau avec la Structure de données imbriquée {#array-join-with-nested-data-structure} - -`ARRAY JOIN` fonctionne également avec [structures de données imbriquées](../../../sql-reference/data-types/nested-data-structures/nested.md): - -``` sql -CREATE TABLE nested_test -( - s String, - nest Nested( - x UInt8, - y UInt32) -) ENGINE = Memory; - -INSERT INTO nested_test -VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []); -``` - -``` text -┌─s───────┬─nest.x──┬─nest.y─────┐ -│ Hello │ [1,2] │ [10,20] │ -│ World │ [3,4,5] │ [30,40,50] │ -│ Goodbye │ [] │ [] │ -└─────────┴─────────┴────────────┘ -``` - -``` sql -SELECT s, `nest.x`, `nest.y` -FROM nested_test -ARRAY JOIN nest; -``` - -``` text -┌─s─────┬─nest.x─┬─nest.y─┐ -│ Hello │ 1 │ 10 │ -│ Hello │ 2 │ 20 │ -│ World │ 3 │ 30 │ -│ World │ 4 │ 40 │ -│ World │ 5 │ 50 │ -└───────┴────────┴────────┘ -``` - -Lorsque vous spécifiez des noms de structures de données imbriquées dans `ARRAY JOIN` le sens est le même que `ARRAY JOIN` avec tous les éléments du tableau qui la compose. Des exemples sont énumérés ci-dessous: - -``` sql -SELECT s, `nest.x`, `nest.y` -FROM nested_test -ARRAY JOIN `nest.x`, `nest.y`; -``` - -``` text -┌─s─────┬─nest.x─┬─nest.y─┐ -│ Hello │ 1 │ 10 │ -│ Hello │ 2 │ 20 │ -│ World │ 3 │ 30 │ -│ World │ 4 │ 40 │ -│ World │ 5 │ 50 │ -└───────┴────────┴────────┘ -``` - -Cette variation a également du sens: - -``` sql -SELECT s, `nest.x`, `nest.y` -FROM nested_test -ARRAY JOIN `nest.x`; -``` - -``` text -┌─s─────┬─nest.x─┬─nest.y─────┐ -│ Hello │ 1 │ [10,20] │ -│ Hello │ 2 │ [10,20] │ -│ World │ 3 │ [30,40,50] │ -│ World │ 4 │ [30,40,50] │ -│ World │ 5 │ [30,40,50] │ -└───────┴────────┴────────────┘ -``` - -Un alias peut être utilisé pour une structure de données imbriquée, afin de sélectionner `JOIN` le résultat ou le tableau source. Exemple: - -``` sql -SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y` -FROM nested_test -ARRAY JOIN nest AS n; -``` - -``` text -┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┐ -│ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ -│ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ -│ World │ 3 │ 30 │ [3,4,5] │ [30,40,50] │ -│ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ -│ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ -└───────┴─────┴─────┴─────────┴────────────┘ -``` - -Exemple d'utilisation de l' [arrayEnumerate](../../../sql-reference/functions/array-functions.md#array_functions-arrayenumerate) fonction: - -``` sql -SELECT s, `n.x`, `n.y`, `nest.x`, `nest.y`, num -FROM nested_test -ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num; -``` - -``` text -┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┬─num─┐ -│ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ 1 │ -│ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ 2 │ -│ World │ 3 │ 30 │ [3,4,5] │ [30,40,50] │ 1 │ -│ World │ 4 │ 40 │ [3,4,5] │ [30,40,50] │ 2 │ -│ World │ 5 │ 50 │ [3,4,5] │ [30,40,50] │ 3 │ -└───────┴─────┴─────┴─────────┴────────────┴─────┘ -``` - -## Détails De Mise En Œuvre {#implementation-details} - -L'ordre d'exécution de la requête est optimisé lors de l'exécution `ARRAY JOIN`. Bien `ARRAY JOIN` doit toujours être spécifié avant l' [WHERE](where.md)/[PREWHERE](prewhere.md) dans une requête, techniquement, ils peuvent être exécutés dans n'importe quel ordre, sauf résultat de `ARRAY JOIN` est utilisé pour le filtrage. L'ordre de traitement est contrôlée par l'optimiseur de requête. diff --git a/docs/fr/sql-reference/statements/select/distinct.md b/docs/fr/sql-reference/statements/select/distinct.md deleted file mode 100644 index 94552018c98..00000000000 --- a/docs/fr/sql-reference/statements/select/distinct.md +++ /dev/null @@ -1,63 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# La Clause DISTINCT {#select-distinct} - -Si `SELECT DISTINCT` est spécifié, seules les lignes uniques restera un résultat de requête. Ainsi, une seule ligne restera hors de tous les ensembles de lignes entièrement correspondantes dans le résultat. - -## Le Traitement Null {#null-processing} - -`DISTINCT` fonctionne avec [NULL](../../syntax.md#null-literal) comme si `NULL` ont une valeur spécifique, et `NULL==NULL`. En d'autres termes, dans le `DISTINCT` résultats, différentes combinaisons avec `NULL` une fois seulement. Elle diffère de `NULL` traitement dans la plupart des autres contextes. - -## Alternative {#alternatives} - -Il est possible d'obtenir le même résultat en appliquant [GROUP BY](group-by.md) sur le même ensemble de valeurs, comme spécifié comme `SELECT` clause, sans utiliser de fonctions d'agrégation. Mais il y a peu de différences de `GROUP BY` approche: - -- `DISTINCT` peut être utilisé avec d' `GROUP BY`. -- Lorsque [ORDER BY](order-by.md) est omis et [LIMIT](limit.md) est définie, la requête s'arrête immédiatement après le nombre de lignes différentes, a été lu. -- Les blocs de données sont produits au fur et à mesure qu'ils sont traités, sans attendre que la requête entière se termine. - -## Limitation {#limitations} - -`DISTINCT` n'est pas pris en charge si `SELECT` a au moins une colonne de tableau. - -## Exemple {#examples} - -Clickhouse prend en charge l'utilisation du `DISTINCT` et `ORDER BY` clauses pour différentes colonnes dans une requête. Le `DISTINCT` la clause est exécutée avant la `ORDER BY` clause. - -Exemple de table: - -``` text -┌─a─┬─b─┐ -│ 2 │ 1 │ -│ 1 │ 2 │ -│ 3 │ 3 │ -│ 2 │ 4 │ -└───┴───┘ -``` - -Lors de la sélection de données avec le `SELECT DISTINCT a FROM t1 ORDER BY b ASC` requête, nous obtenons le résultat suivant: - -``` text -┌─a─┐ -│ 2 │ -│ 1 │ -│ 3 │ -└───┘ -``` - -Si nous changeons la direction de tri `SELECT DISTINCT a FROM t1 ORDER BY b DESC`, nous obtenons le résultat suivant: - -``` text -┌─a─┐ -│ 3 │ -│ 1 │ -│ 2 │ -└───┘ -``` - -Rangée `2, 4` a été coupé avant de les trier. - -Prenez en compte cette spécificité d'implémentation lors de la programmation des requêtes. diff --git a/docs/fr/sql-reference/statements/select/format.md b/docs/fr/sql-reference/statements/select/format.md deleted file mode 100644 index a88bb7831ba..00000000000 --- a/docs/fr/sql-reference/statements/select/format.md +++ /dev/null @@ -1,18 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# FORMAT de la Clause {#format-clause} - -Clickhouse prend en charge une large gamme de [formats de sérialisation](../../../interfaces/formats.md) qui peut être utilisé sur les résultats de la requête entre autres choses. Il existe plusieurs façons de choisir un format pour `SELECT` de sortie, l'un d'eux est de spécifier `FORMAT format` à la fin de la requête pour obtenir les données résultantes dans tout format spécifique. - -Un format spécifique peut être utilisé pour des raisons de commodité, d'intégration avec d'autres systèmes ou d'amélioration des performances. - -## Format Par Défaut {#default-format} - -Si l' `FORMAT` la clause est omise, le format par défaut est utilisé, ce qui dépend à la fois des paramètres et de l'interface utilisée pour accéder au serveur ClickHouse. Pour l' [Interface HTTP](../../../interfaces/http.md) et la [client de ligne de commande](../../../interfaces/cli.md) en mode batch, le format par défaut est `TabSeparated`. Pour le client de ligne de commande en mode interactif, le format par défaut est `PrettyCompact` (il produit des tables compactes lisibles par l'homme). - -## Détails De Mise En Œuvre {#implementation-details} - -Lors de l'utilisation du client de ligne de commande, les données sont toujours transmises sur le réseau dans un format efficace interne (`Native`). Le client interprète indépendamment le `FORMAT` clause de la requête et formate les données elles-mêmes (soulageant ainsi le réseau et le serveur de la charge supplémentaire). diff --git a/docs/fr/sql-reference/statements/select/from.md b/docs/fr/sql-reference/statements/select/from.md deleted file mode 100644 index 964ffdd13fb..00000000000 --- a/docs/fr/sql-reference/statements/select/from.md +++ /dev/null @@ -1,44 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# De la Clause {#select-from} - -Le `FROM` clause spécifie la source à partir de laquelle lire les données: - -- [Table](../../../engines/table-engines/index.md) -- [Sous-requête](index.md) {##TODO: meilleur lien ##} -- [Fonction de Table](../../table-functions/index.md#table-functions) - -[JOIN](join.md) et [ARRAY JOIN](array-join.md) les clauses peuvent également être utilisées pour étendre la fonctionnalité de la `FROM` clause. - -Subquery est un autre `SELECT` requête qui peut être spécifié entre parenthèses à l'intérieur `FROM` clause. - -`FROM` la clause peut contenir plusieurs sources de données, séparées par des virgules, ce qui équivaut à effectuer [CROSS JOIN](join.md) sur eux. - -## Modificateur FINAL {#select-from-final} - -Lorsque `FINAL` est spécifié, ClickHouse fusionne complètement les données avant de renvoyer le résultat et effectue ainsi toutes les transformations de données qui se produisent lors des fusions pour le moteur de table donné. - -Il est applicable lors de la sélection de données à partir de tables qui utilisent [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-la famille de moteurs (à l'exception de `GraphiteMergeTree`). Également pris en charge pour: - -- [Répliqué](../../../engines/table-engines/mergetree-family/replication.md) les versions de `MergeTree` moteur. -- [Vue](../../../engines/table-engines/special/view.md), [Tampon](../../../engines/table-engines/special/buffer.md), [Distribué](../../../engines/table-engines/special/distributed.md), et [MaterializedView](../../../engines/table-engines/special/materializedview.md) moteurs qui fonctionnent sur d'autres moteurs, à condition qu'ils aient été créés sur `MergeTree`-tables de moteur. - -### Inconvénient {#drawbacks} - -Requêtes qui utilisent `FINAL` sont exécutés pas aussi vite que les requêtes similaires qui ne le font pas, car: - -- La requête est exécutée dans un seul thread et les données sont fusionnées lors de l'exécution de la requête. -- Les requêtes avec `FINAL` lire les colonnes de clé primaire en plus des colonnes spécifiées dans la requête. - -**Dans la plupart des cas, évitez d'utiliser `FINAL`.** L'approche commune consiste à utiliser différentes requêtes qui supposent les processus d'arrière-plan du `MergeTree` le moteur n'est pas encore arrivé et y faire face en appliquant l'agrégation (par exemple, pour éliminer les doublons). {##TODO: exemples ##} - -## Détails De Mise En Œuvre {#implementation-details} - -Si l' `FROM` la clause est omise, les données seront lues à partir `system.one` table. -Le `system.one` table contient exactement une ligne (cette table remplit le même but que la table double trouvée dans d'autres SGBD). - -Pour exécuter une requête, toutes les colonnes mentionnées dans la requête sont extraites de la table appropriée. Toutes les colonnes non nécessaires pour la requête externe sont rejetées des sous-requêtes. -Si une requête ne répertorie aucune colonne (par exemple, `SELECT count() FROM t`), une colonne est extraite de la table de toute façon (la plus petite est préférée), afin de calculer le nombre de lignes. diff --git a/docs/fr/sql-reference/statements/select/group-by.md b/docs/fr/sql-reference/statements/select/group-by.md deleted file mode 100644 index 9d1b5c276d5..00000000000 --- a/docs/fr/sql-reference/statements/select/group-by.md +++ /dev/null @@ -1,132 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause GROUP BY {#select-group-by-clause} - -`GROUP BY` la clause change le `SELECT` requête dans un mode d'agrégation, qui fonctionne comme suit: - -- `GROUP BY` clause contient une liste des expressions (ou une seule expression, qui est considéré comme la liste de longueur). Cette liste agit comme un “grouping key”, tandis que chaque expression individuelle sera appelée “key expressions”. -- Toutes les expressions dans le [SELECT](index.md), [HAVING](having.md), et [ORDER BY](order-by.md) clause **devoir** être calculé sur la base d'expressions clés **ou** sur [les fonctions d'agrégation](../../../sql-reference/aggregate-functions/index.md) sur les expressions non-clés (y compris les colonnes simples). En d'autres termes, chaque colonne sélectionnée dans la table doit être utilisée soit dans une expression de clé, soit dans une fonction d'agrégat, mais pas les deux. -- Résultat de l'agrégation de `SELECT` la requête contiendra autant de lignes qu'il y avait des valeurs uniques de “grouping key” dans la table source. Habituellement, cela réduit considérablement le nombre de lignes, souvent par ordre de grandeur, mais pas nécessairement: le nombre de lignes reste le même si tous “grouping key” les valeurs sont distinctes. - -!!! note "Note" - Il existe un moyen supplémentaire d'exécuter l'agrégation sur une table. Si une requête ne contient que des colonnes de table à l'intérieur des fonctions `GROUP BY clause` peut être omis, et l'agrégation par un ensemble vide de touches est supposé. Ces interrogations renvoient toujours exactement une ligne. - -## Le Traitement NULL {#null-processing} - -Pour le regroupement, ClickHouse interprète [NULL](../../syntax.md#null-literal) comme une valeur, et `NULL==NULL`. Elle diffère de `NULL` traitement dans la plupart des autres contextes. - -Voici un exemple pour montrer ce que cela signifie. - -Supposons que vous avez cette table: - -``` text -┌─x─┬────y─┐ -│ 1 │ 2 │ -│ 2 │ ᴺᵁᴸᴸ │ -│ 3 │ 2 │ -│ 3 │ 3 │ -│ 3 │ ᴺᵁᴸᴸ │ -└───┴──────┘ -``` - -Requête `SELECT sum(x), y FROM t_null_big GROUP BY y` résultats dans: - -``` text -┌─sum(x)─┬────y─┐ -│ 4 │ 2 │ -│ 3 │ 3 │ -│ 5 │ ᴺᵁᴸᴸ │ -└────────┴──────┘ -``` - -Vous pouvez voir que `GROUP BY` pour `y = NULL` résumer `x` comme si `NULL` a cette valeur. - -Si vous passez plusieurs clés `GROUP BY` le résultat vous donnera toutes les combinaisons de la sélection, comme si `NULL` ont une valeur spécifique. - -## Avec modificateur de totaux {#with-totals-modifier} - -Si l' `WITH TOTALS` modificateur est spécifié, une autre ligne sera calculée. Cette ligne aura des colonnes clés contenant des valeurs par défaut (zéros ou lignes vides), et des colonnes de fonctions d'agrégat avec les valeurs calculées sur toutes les lignes (le “total” valeur). - -Cette ligne supplémentaire est uniquement produite en `JSON*`, `TabSeparated*`, et `Pretty*` formats, séparément des autres lignes: - -- Dans `JSON*` formats, cette ligne est sortie en tant que distinct ‘totals’ champ. -- Dans `TabSeparated*` formats, la ligne vient après le résultat principal, précédé par une ligne vide (après les autres données). -- Dans `Pretty*` formats, la ligne est sortie comme une table séparée après le résultat principal. -- Dans les autres formats, il n'est pas disponible. - -`WITH TOTALS` peut être exécuté de différentes manières lorsqu'il est présent. Le comportement dépend de l' ‘totals_mode’ paramètre. - -### Configuration Du Traitement Des Totaux {#configuring-totals-processing} - -Par défaut, `totals_mode = 'before_having'`. Dans ce cas, ‘totals’ est calculé sur toutes les lignes, y compris celles qui ne passent pas par `max_rows_to_group_by`. - -Les autres alternatives incluent uniquement les lignes qui passent à travers avoir dans ‘totals’, et se comporter différemment avec le réglage `max_rows_to_group_by` et `group_by_overflow_mode = 'any'`. - -`after_having_exclusive` – Don't include rows that didn't pass through `max_rows_to_group_by`. En d'autres termes, ‘totals’ aura moins ou le même nombre de lignes que si `max_rows_to_group_by` ont été omis. - -`after_having_inclusive` – Include all the rows that didn't pass through ‘max_rows_to_group_by’ dans ‘totals’. En d'autres termes, ‘totals’ aura plus ou le même nombre de lignes que si `max_rows_to_group_by` ont été omis. - -`after_having_auto` – Count the number of rows that passed through HAVING. If it is more than a certain amount (by default, 50%), include all the rows that didn't pass through ‘max_rows_to_group_by’ dans ‘totals’. Sinon, ne pas les inclure. - -`totals_auto_threshold` – By default, 0.5. The coefficient for `after_having_auto`. - -Si `max_rows_to_group_by` et `group_by_overflow_mode = 'any'` ne sont pas utilisés, toutes les variations de `after_having` sont les mêmes, et vous pouvez utiliser l'un d'eux (par exemple, `after_having_auto`). - -Vous pouvez utiliser avec les totaux dans les sous-requêtes, y compris les sous-requêtes dans la clause JOIN (dans ce cas, les valeurs totales respectives sont combinées). - -## Exemple {#examples} - -Exemple: - -``` sql -SELECT - count(), - median(FetchTiming > 60 ? 60 : FetchTiming), - count() - sum(Refresh) -FROM hits -``` - -Cependant, contrairement au SQL standard, si la table n'a pas de lignes (soit il n'y en a pas du tout, soit il n'y en a pas après avoir utilisé WHERE to filter), un résultat vide est renvoyé, et non le résultat d'une des lignes contenant les valeurs initiales des fonctions d'agrégat. - -Contrairement à MySQL (et conforme à SQL standard), vous ne pouvez pas obtenir une valeur d'une colonne qui n'est pas dans une fonction clé ou agrégée (sauf les expressions constantes). Pour contourner ce problème, vous pouvez utiliser le ‘any’ fonction d'agrégation (récupère la première valeur rencontrée) ou ‘min/max’. - -Exemple: - -``` sql -SELECT - domainWithoutWWW(URL) AS domain, - count(), - any(Title) AS title -- getting the first occurred page header for each domain. -FROM hits -GROUP BY domain -``` - -Pour chaque valeur de clé différente rencontrée, GROUP BY calcule un ensemble de valeurs de fonction d'agrégation. - -GROUP BY n'est pas pris en charge pour les colonnes de tableau. - -Une constante ne peut pas être spécifiée comme arguments pour les fonctions d'agrégation. Exemple: somme(1). Au lieu de cela, vous pouvez vous débarrasser de la constante. Exemple: `count()`. - -## Détails De Mise En Œuvre {#implementation-details} - -L'agrégation est l'une des caractéristiques les plus importantes d'un SGBD orienté colonne, et donc son implémentation est l'une des parties les plus optimisées de ClickHouse. Par défaut, l'agrégation se fait en mémoire à l'aide d'une table de hachage. Il a plus de 40 spécialisations qui sont choisies automatiquement en fonction de “grouping key” types de données. - -### Groupe par dans la mémoire externe {#select-group-by-in-external-memory} - -Vous pouvez activer le dumping des données temporaires sur le disque pour limiter l'utilisation de la mémoire pendant `GROUP BY`. -Le [max_bytes_before_external_group_by](../../../operations/settings/settings.md#settings-max_bytes_before_external_group_by) réglage détermine le seuil de consommation de RAM pour le dumping `GROUP BY` données temporaires dans le système de fichiers. Si elle est définie sur 0 (valeur par défaut), elle est désactivée. - -Lors de l'utilisation de `max_bytes_before_external_group_by`, nous vous recommandons de définir `max_memory_usage` environ deux fois plus élevé. Ceci est nécessaire car il y a deux étapes à l'agrégation: la lecture des données et la formation des données intermédiaires (1) et la fusion des données intermédiaires (2). Le Dumping des données dans le système de fichiers ne peut se produire qu'au cours de l'étape 1. Si les données temporaires n'ont pas été vidées, l'étape 2 peut nécessiter jusqu'à la même quantité de mémoire qu'à l'étape 1. - -Par exemple, si [max_memory_usage](../../../operations/settings/settings.md#settings_max_memory_usage) a été défini sur 10000000000 et que vous souhaitez utiliser l'agrégation externe, il est logique de définir `max_bytes_before_external_group_by` à 10000000000, et `max_memory_usage` à 20000000000. Lorsque l'agrégation externe est déclenchée (s'il y a eu au moins un vidage de données temporaires), la consommation maximale de RAM n'est que légèrement supérieure à `max_bytes_before_external_group_by`. - -Avec le traitement des requêtes distribuées, l'agrégation externe est effectuée sur des serveurs distants. Pour que le serveur demandeur n'utilise qu'une petite quantité de RAM, définissez `distributed_aggregation_memory_efficient` 1. - -Lors de la fusion de données vidées sur le disque, ainsi que lors de la fusion des résultats de serveurs distants lorsque `distributed_aggregation_memory_efficient` paramètre est activé, consomme jusqu'à `1/256 * the_number_of_threads` à partir de la quantité totale de mémoire RAM. - -Lorsque l'agrégation externe est activée, s'il y a moins de `max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). - -Si vous avez un [ORDER BY](order-by.md) avec un [LIMIT](limit.md) après `GROUP BY` puis la quantité de RAM dépend de la quantité de données dans `LIMIT`, pas dans l'ensemble de la table. Mais si l' `ORDER BY` n'a pas `LIMIT`, n'oubliez pas d'activer externe de tri (`max_bytes_before_external_sort`). diff --git a/docs/fr/sql-reference/statements/select/having.md b/docs/fr/sql-reference/statements/select/having.md deleted file mode 100644 index 9425830c3d4..00000000000 --- a/docs/fr/sql-reference/statements/select/having.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause HAVING {#having-clause} - -Permet de filtrer les résultats d'agrégation produits par [GROUP BY](group-by.md). Il est similaire à la [WHERE](where.md) la clause, mais la différence est que `WHERE` est effectuée avant l'agrégation, tandis que `HAVING` est effectué d'après elle. - -Il est possible de référencer les résultats d'agrégation à partir de `SELECT` la clause dans `HAVING` clause par leur alias. Alternativement, `HAVING` clause peut filtrer sur les résultats d'agrégats supplémentaires qui ne sont pas retournés dans les résultats de la requête. - -## Limitation {#limitations} - -`HAVING` ne peut pas être utilisé si le regroupement n'est pas effectuée. Utiliser `WHERE` plutôt. diff --git a/docs/fr/sql-reference/statements/select/index.md b/docs/fr/sql-reference/statements/select/index.md deleted file mode 100644 index 1d53ae80eb4..00000000000 --- a/docs/fr/sql-reference/statements/select/index.md +++ /dev/null @@ -1,158 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 33 -toc_title: SELECT ---- - -# Sélectionnez la syntaxe des requêtes {#select-queries-syntax} - -`SELECT` effectue la récupération des données. - -``` sql -[WITH expr_list|(subquery)] -SELECT [DISTINCT] expr_list -[FROM [db.]table | (subquery) | table_function] [FINAL] -[SAMPLE sample_coeff] -[ARRAY JOIN ...] -[GLOBAL] [ANY|ALL|ASOF] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI] JOIN (subquery)|table (ON )|(USING ) -[PREWHERE expr] -[WHERE expr] -[GROUP BY expr_list] [WITH TOTALS] -[HAVING expr] -[ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] -[LIMIT [offset_value, ]n BY columns] -[LIMIT [n, ]m] [WITH TIES] -[UNION ALL ...] -[INTO OUTFILE filename] -[FORMAT format] -``` - -Toutes les clauses sont facultatives, à l'exception de la liste d'expressions requise immédiatement après `SELECT` qui est abordée plus en détail [dessous](#select-clause). - -Spécificités de chaque clause facultative, sont couverts dans des sections distinctes, qui sont énumérés dans le même ordre qu'elles sont exécutées: - -- [AVEC la clause](with.md) -- [La clause DISTINCT](distinct.md) -- [De la clause](from.md) -- [Exemple de clause](sample.md) -- [Clause de JOINTURE](join.md) -- [Clause PREWHERE](prewhere.md) -- [Clause where](where.md) -- [Groupe par clause](group-by.md) -- [Limite par clause](limit-by.md) -- [Clause HAVING](having.md) -- [Clause SELECT](#select-clause) -- [Clause LIMIT](limit.md) -- [Clause UNION ALL](union.md) - -## Clause SELECT {#select-clause} - -[Expression](../../syntax.md#syntax-expressions) spécifié dans le `SELECT` clause sont calculés après toutes les opérations dans les clauses décrites ci-dessus sont terminés. Ces expressions fonctionnent comme si elles s'appliquaient à des lignes séparées dans le résultat. Si les expressions dans le `SELECT` la clause contient des fonctions d'agrégation, puis clickhouse traite les fonctions d'agrégation et les expressions utilisées [GROUP BY](group-by.md) agrégation. - -Si vous souhaitez inclure toutes les colonnes dans le résultat, utilisez l'astérisque (`*`) symbole. Exemple, `SELECT * FROM ...`. - -Pour correspondre à certaines colonnes dans le résultat avec un [re2](https://en.wikipedia.org/wiki/RE2_(software)) expression régulière, vous pouvez utiliser le `COLUMNS` expression. - -``` sql -COLUMNS('regexp') -``` - -Par exemple, considérez le tableau: - -``` sql -CREATE TABLE default.col_names (aa Int8, ab Int8, bc Int8) ENGINE = TinyLog -``` - -La requête suivante sélectionne les données de toutes les colonnes contenant les `a` symbole dans leur nom. - -``` sql -SELECT COLUMNS('a') FROM col_names -``` - -``` text -┌─aa─┬─ab─┐ -│ 1 │ 1 │ -└────┴────┘ -``` - -Les colonnes sélectionnées sont retournés pas dans l'ordre alphabétique. - -Vous pouvez utiliser plusieurs `COLUMNS` expressions dans une requête et leur appliquer des fonctions. - -Exemple: - -``` sql -SELECT COLUMNS('a'), COLUMNS('c'), toTypeName(COLUMNS('c')) FROM col_names -``` - -``` text -┌─aa─┬─ab─┬─bc─┬─toTypeName(bc)─┐ -│ 1 │ 1 │ 1 │ Int8 │ -└────┴────┴────┴────────────────┘ -``` - -Chaque colonne renvoyée par le `COLUMNS` expression est passée à la fonction en tant qu'argument séparé. Vous pouvez également passer d'autres arguments à la fonction si elle les supporte. Soyez prudent lorsque vous utilisez des fonctions. Si une fonction ne prend pas en charge le nombre d'arguments que vous lui avez transmis, ClickHouse lève une exception. - -Exemple: - -``` sql -SELECT COLUMNS('a') + COLUMNS('c') FROM col_names -``` - -``` text -Received exception from server (version 19.14.1): -Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of arguments for function plus doesn't match: passed 3, should be 2. -``` - -Dans cet exemple, `COLUMNS('a')` retourne deux colonnes: `aa` et `ab`. `COLUMNS('c')` renvoie la `bc` colonne. Le `+` l'opérateur ne peut pas s'appliquer à 3 arguments, donc ClickHouse lève une exception avec le message pertinent. - -Colonnes qui correspondent à la `COLUMNS` l'expression peut avoir différents types de données. Si `COLUMNS` ne correspond à aucune colonne et est la seule expression dans `SELECT`, ClickHouse lance une exception. - -### Astérisque {#asterisk} - -Vous pouvez mettre un astérisque dans quelque partie de la requête au lieu d'une expression. Lorsque la requête est analysée, l'astérisque est étendu à une liste de toutes les colonnes `MATERIALIZED` et `ALIAS` colonne). Il n'y a que quelques cas où l'utilisation d'un astérisque est justifiée: - -- Lors de la création d'un vidage de table. -- Pour les tables contenant seulement quelques colonnes, comme les tables système. -- Pour obtenir des informations sur ce que sont les colonnes dans une table. Dans ce cas, la valeur `LIMIT 1`. Mais il est préférable d'utiliser la `DESC TABLE` requête. -- Quand il y a une forte filtration sur un petit nombre de colonnes en utilisant `PREWHERE`. -- Dans les sous-requêtes (puisque les colonnes qui ne sont pas nécessaires pour la requête externe sont exclues des sous-requêtes). - -Dans tous les autres cas, nous ne recommandons pas d'utiliser l'astérisque, car il ne vous donne que les inconvénients d'un SGBD colonnaire au lieu des avantages. En d'autres termes, l'utilisation de l'astérisque n'est pas recommandée. - -### Les Valeurs Extrêmes {#extreme-values} - -En plus des résultats, vous pouvez également obtenir des valeurs minimales et maximales pour les colonnes de résultats. Pour ce faire, définissez la **extrême** réglage sur 1. Les Minimums et les maximums sont calculés pour les types numériques, les dates et les dates avec des heures. Pour les autres colonnes, les valeurs par défaut sont sorties. - -An extra two rows are calculated – the minimums and maximums, respectively. These extra two rows are output in `JSON*`, `TabSeparated*`, et `Pretty*` [format](../../../interfaces/formats.md), séparés des autres lignes. Ils ne sont pas Produits pour d'autres formats. - -Dans `JSON*` formats, les valeurs extrêmes sont sorties dans un ‘extremes’ champ. Dans `TabSeparated*` formats, la ligne vient après le résultat principal, et après ‘totals’ si elle est présente. Elle est précédée par une ligne vide (après les autres données). Dans `Pretty*` formats, la ligne est sortie comme une table séparée après le résultat principal, et après `totals` si elle est présente. - -Les valeurs extrêmes sont calculées pour les lignes avant `LIMIT` mais après `LIMIT BY`. Cependant, lors de l'utilisation de `LIMIT offset, size`, les lignes avant de les `offset` sont inclus dans `extremes`. Dans les requêtes de flux, le résultat peut également inclure un petit nombre de lignes qui ont traversé `LIMIT`. - -### Note {#notes} - -Vous pouvez utiliser des synonymes (`AS` alias) dans n'importe quelle partie d'une requête. - -Le `GROUP BY` et `ORDER BY` les clauses ne supportent pas les arguments positionnels. Cela contredit MySQL, mais est conforme à SQL standard. Exemple, `GROUP BY 1, 2` will be interpreted as grouping by constants (i.e. aggregation of all rows into one). - -## Détails De Mise En Œuvre {#implementation-details} - -Si la requête omet le `DISTINCT`, `GROUP BY` et `ORDER BY` les clauses et les `IN` et `JOIN` sous-requêtes, la requête sera complètement traitée en flux, en utilisant O (1) quantité de RAM. Sinon, la requête peut consommer beaucoup de RAM si les restrictions appropriées ne sont pas spécifiées: - -- `max_memory_usage` -- `max_rows_to_group_by` -- `max_rows_to_sort` -- `max_rows_in_distinct` -- `max_bytes_in_distinct` -- `max_rows_in_set` -- `max_bytes_in_set` -- `max_rows_in_join` -- `max_bytes_in_join` -- `max_bytes_before_external_sort` -- `max_bytes_before_external_group_by` - -Pour plus d'informations, consultez la section “Settings”. Il est possible d'utiliser le tri externe (sauvegarde des tables temporaires sur un disque) et l'agrégation externe. - -{## [Article Original](https://clickhouse.tech/docs/en/sql-reference/statements/select/) ##} diff --git a/docs/fr/sql-reference/statements/select/into-outfile.md b/docs/fr/sql-reference/statements/select/into-outfile.md deleted file mode 100644 index 0150de7cb97..00000000000 --- a/docs/fr/sql-reference/statements/select/into-outfile.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Dans OUTFILE Clause {#into-outfile-clause} - -Ajouter l' `INTO OUTFILE filename` clause (où filename est un littéral de chaîne) pour `SELECT query` pour rediriger sa sortie vers le fichier spécifié côté client. - -## Détails De Mise En Œuvre {#implementation-details} - -- Cette fonctionnalité est disponible dans les [client de ligne de commande](../../../interfaces/cli.md) et [clickhouse-local](../../../operations/utilities/clickhouse-local.md). Ainsi, une requête envoyée par [Interface HTTP](../../../interfaces/http.md) va échouer. -- La requête échouera si un fichier portant le même nom existe déjà. -- Défaut [le format de sortie](../../../interfaces/formats.md) être `TabSeparated` (comme dans le mode batch client en ligne de commande). diff --git a/docs/fr/sql-reference/statements/select/join.md b/docs/fr/sql-reference/statements/select/join.md deleted file mode 100644 index 4233a120674..00000000000 --- a/docs/fr/sql-reference/statements/select/join.md +++ /dev/null @@ -1,187 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause de JOINTURE {#select-join} - -Join produit une nouvelle table en combinant des colonnes d'une ou plusieurs tables en utilisant des valeurs communes à chacune. C'est une opération courante dans les bases de données avec support SQL, ce qui correspond à [l'algèbre relationnelle](https://en.wikipedia.org/wiki/Relational_algebra#Joins_and_join-like_operators) rejoindre. Le cas particulier d'une jointure de table est souvent appelé “self-join”. - -Syntaxe: - -``` sql -SELECT -FROM -[GLOBAL] [INNER|LEFT|RIGHT|FULL|CROSS] [OUTER|SEMI|ANTI|ANY|ASOF] JOIN -(ON )|(USING ) ... -``` - -Les Expressions de `ON` clause et colonnes de `USING` clause sont appelés “join keys”. Sauf indication contraire, joindre un produit [Produit cartésien](https://en.wikipedia.org/wiki/Cartesian_product) des lignes, avec correspondance “join keys”, ce qui pourrait produire des résultats avec beaucoup plus de lignes que les tables source. - -## Types de jointure pris en charge {#select-join-types} - -Tous les standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) les types sont pris en charge: - -- `INNER JOIN`, seules les lignes correspondantes sont retournés. -- `LEFT OUTER JOIN`, les lignes non correspondantes de la table de gauche sont retournées en plus des lignes correspondantes. -- `RIGHT OUTER JOIN`, les lignes non correspondantes de la table de gauche sont retournées en plus des lignes correspondantes. -- `FULL OUTER JOIN`, les lignes non correspondantes des deux tables sont renvoyées en plus des lignes correspondantes. -- `CROSS JOIN`, produit le produit cartésien des tables entières, “join keys” être **pas** défini. - -`JOIN` sans type spécifié implique `INNER`. Mot `OUTER` peut les oublier. Syntaxe Alternative pour `CROSS JOIN` spécifie plusieurs tables dans [De la clause](from.md) séparés par des virgules. - -Autres types de jointure disponibles dans ClickHouse: - -- `LEFT SEMI JOIN` et `RIGHT SEMI JOIN` une liste blanche sur “join keys”, sans produire un produit cartésien. -- `LEFT ANTI JOIN` et `RIGHT ANTI JOIN` une liste noire sur “join keys”, sans produire un produit cartésien. -- `LEFT ANY JOIN`, `RIGHT ANY JOIN` et `INNER ANY JOIN`, partially (for opposite side of `LEFT` and `RIGHT`) or completely (for `INNER` and `FULL`) disables the cartesian product for standard `JOIN` types. -- `ASOF JOIN` et `LEFT ASOF JOIN`, joining sequences with a non-exact match. `ASOF JOIN` usage is described below. - -## Setting {#join-settings} - -!!! note "Note" - La valeur de rigueur par défaut peut être remplacée à l'aide [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) paramètre. - -### ASOF joindre L'utilisation {#asof-join-usage} - -`ASOF JOIN` est utile lorsque vous devez joindre des enregistrements qui n'ont pas de correspondance exacte. - -Tables pour `ASOF JOIN` doit avoir une colonne de séquence ordonnée. Cette colonne ne peut pas être seule dans une table et doit être l'un des types de données: `UInt32`, `UInt64`, `Float32`, `Float64`, `Date`, et `DateTime`. - -Syntaxe `ASOF JOIN ... ON`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF LEFT JOIN table_2 -ON equi_cond AND closest_match_cond -``` - -Vous pouvez utiliser n'importe quel nombre de conditions d'égalité et exactement une condition de correspondance la plus proche. Exemple, `SELECT count() FROM table_1 ASOF LEFT JOIN table_2 ON table_1.a == table_2.b AND table_2.t <= table_1.t`. - -Conditions prises en charge pour la correspondance la plus proche: `>`, `>=`, `<`, `<=`. - -Syntaxe `ASOF JOIN ... USING`: - -``` sql -SELECT expressions_list -FROM table_1 -ASOF JOIN table_2 -USING (equi_column1, ... equi_columnN, asof_column) -``` - -`ASOF JOIN` utiliser `equi_columnX` pour rejoindre sur l'égalité et `asof_column` pour rejoindre le match le plus proche avec le `table_1.asof_column >= table_2.asof_column` condition. Le `asof_column` colonne toujours la dernière dans le `USING` clause. - -Par exemple, considérez les tableaux suivants: - - table_1 table_2 - event | ev_time | user_id event | ev_time | user_id - ----------|---------|---------- ----------|---------|---------- - ... ... - event_1_1 | 12:00 | 42 event_2_1 | 11:59 | 42 - ... event_2_2 | 12:30 | 42 - event_1_2 | 13:00 | 42 event_2_3 | 13:00 | 42 - ... ... - -`ASOF JOIN` peut prendre la date d'un événement utilisateur de `table_1` et trouver un événement dans `table_2` où le timestamp est plus proche de l'horodatage de l'événement à partir de `table_1` correspondant à la condition de correspondance la plus proche. Les valeurs d'horodatage égales sont les plus proches si elles sont disponibles. Ici, l' `user_id` la colonne peut être utilisée pour joindre sur l'égalité et le `ev_time` la colonne peut être utilisée pour se joindre à la correspondance la plus proche. Dans notre exemple, `event_1_1` peut être jointe à `event_2_1` et `event_1_2` peut être jointe à `event_2_3`, mais `event_2_2` ne peut pas être rejoint. - -!!! note "Note" - `ASOF` jointure est **pas** pris en charge dans le [Rejoindre](../../../engines/table-engines/special/join.md) tableau moteur. - -## Jointure Distribuée {#global-join} - -Il existe deux façons d'exécuter join impliquant des tables distribuées: - -- Lors de l'utilisation normale `JOIN` la requête est envoyée aux serveurs distants. Les sous-requêtes sont exécutées sur chacune d'elles afin de créer la bonne table, et la jointure est effectuée avec cette table. En d'autres termes, la table de droite est formée sur chaque serveur séparément. -- Lors de l'utilisation de `GLOBAL ... JOIN`, d'abord le serveur demandeur exécute une sous-requête pour calculer la bonne table. Cette table temporaire est transmise à chaque serveur distant, et les requêtes sont exécutées sur eux en utilisant les données temporaires qui ont été transmises. - -Soyez prudent lorsque vous utilisez `GLOBAL`. Pour plus d'informations, voir le [Sous-requêtes distribuées](../../operators/in.md#select-distributed-subqueries) section. - -## Recommandations D'Utilisation {#usage-recommendations} - -### Traitement des cellules vides ou nulles {#processing-of-empty-or-null-cells} - -Lors de la jonction de tables, les cellules vides peuvent apparaître. Paramètre [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) définir comment clickhouse remplit ces cellules. - -Si l' `JOIN` les touches sont [Nullable](../../data-types/nullable.md) champs, les lignes où au moins une des clés a la valeur [NULL](../../../sql-reference/syntax.md#null-literal) ne sont pas jointes. - -### Syntaxe {#syntax} - -Les colonnes spécifiées dans `USING` doit avoir les mêmes noms dans les deux sous-requêtes, et les autres colonnes doivent être nommées différemment. Vous pouvez utiliser des alias pour les noms des colonnes dans les sous-requêtes. - -Le `USING` clause spécifie une ou plusieurs colonnes de jointure, qui établit l'égalité de ces colonnes. La liste des colonnes est définie sans crochets. Les conditions de jointure plus complexes ne sont pas prises en charge. - -### Limitations De Syntaxe {#syntax-limitations} - -Pour plusieurs `JOIN` clauses dans un seul `SELECT` requête: - -- Prendre toutes les colonnes via `*` n'est disponible que si les tables sont jointes, pas les sous-requêtes. -- Le `PREWHERE` la clause n'est pas disponible. - -Pour `ON`, `WHERE`, et `GROUP BY` clause: - -- Les expressions arbitraires ne peuvent pas être utilisées dans `ON`, `WHERE`, et `GROUP BY` mais vous pouvez définir une expression dans un `SELECT` clause et ensuite l'utiliser dans ces clauses via un alias. - -### Performance {#performance} - -Lors de l'exécution d'un `JOIN`, il n'y a pas d'optimisation de la commande d'exécution par rapport aux autres stades de la requête. La jointure (une recherche dans la table de droite) est exécutée avant de filtrer `WHERE` et avant l'agrégation. - -Chaque fois qu'une requête est exécutée avec la même `JOIN`, la sous-requête est exécutée à nouveau car le résultat n'est pas mis en cache. Pour éviter cela, utilisez la spéciale [Rejoindre](../../../engines/table-engines/special/join.md) table engine, qui est un tableau préparé pour l'assemblage qui est toujours en RAM. - -Dans certains cas, il est plus efficace d'utiliser [IN](../../operators/in.md) plutôt `JOIN`. - -Si vous avez besoin d'un `JOIN` pour se joindre à des tables de dimension (ce sont des tables relativement petites qui contiennent des propriétés de dimension, telles que des noms pour des campagnes publicitaires), un `JOIN` peut-être pas très pratique en raison du fait que la bonne table est ré-accédée pour chaque requête. Pour de tels cas, il y a un “external dictionaries” la fonctionnalité que vous devez utiliser à la place de `JOIN`. Pour plus d'informations, voir le [Dictionnaires externes](../../dictionaries/external-dictionaries/external-dicts.md) section. - -### Limitations De Mémoire {#memory-limitations} - -Par défaut, ClickHouse utilise [jointure de hachage](https://en.wikipedia.org/wiki/Hash_join) algorithme. ClickHouse prend le `` et crée une table de hachage pour cela dans la RAM. Après un certain seuil de consommation de mémoire, ClickHouse revient à fusionner l'algorithme de jointure. - -Si vous devez restreindre la consommation de mémoire de l'opération join utilisez les paramètres suivants: - -- [max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) — Limits number of rows in the hash table. -- [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) — Limits size of the hash table. - -Lorsque l'une de ces limites est atteinte, ClickHouse agit comme [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) réglage des instructions. - -## Exemple {#examples} - -Exemple: - -``` sql -SELECT - CounterID, - hits, - visits -FROM -( - SELECT - CounterID, - count() AS hits - FROM test.hits - GROUP BY CounterID -) ANY LEFT JOIN -( - SELECT - CounterID, - sum(Sign) AS visits - FROM test.visits - GROUP BY CounterID -) USING CounterID -ORDER BY hits DESC -LIMIT 10 -``` - -``` text -┌─CounterID─┬───hits─┬─visits─┐ -│ 1143050 │ 523264 │ 13665 │ -│ 731962 │ 475698 │ 102716 │ -│ 722545 │ 337212 │ 108187 │ -│ 722889 │ 252197 │ 10547 │ -│ 2237260 │ 196036 │ 9522 │ -│ 23057320 │ 147211 │ 7689 │ -│ 722818 │ 90109 │ 17847 │ -│ 48221 │ 85379 │ 4652 │ -│ 19762435 │ 77807 │ 7026 │ -│ 722884 │ 77492 │ 11056 │ -└───────────┴────────┴────────┘ -``` diff --git a/docs/fr/sql-reference/statements/select/limit-by.md b/docs/fr/sql-reference/statements/select/limit-by.md deleted file mode 100644 index 4d1bd766ef1..00000000000 --- a/docs/fr/sql-reference/statements/select/limit-by.md +++ /dev/null @@ -1,71 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Limite par Clause {#limit-by-clause} - -Une requête avec l' `LIMIT n BY expressions` la clause sélectionne le premier `n` lignes pour chaque valeur distincte de `expressions`. La clé pour `LIMIT BY` peut contenir n'importe quel nombre de [expression](../../syntax.md#syntax-expressions). - -ClickHouse prend en charge les variantes de syntaxe suivantes: - -- `LIMIT [offset_value, ]n BY expressions` -- `LIMIT n OFFSET offset_value BY expressions` - -Pendant le traitement de la requête, ClickHouse sélectionne les données classées par clé de tri. La clé de tri est définie explicitement à l'aide [ORDER BY](order-by.md) clause ou implicitement en tant que propriété du moteur de table. Puis clickhouse s'applique `LIMIT n BY expressions` et renvoie le premier `n` lignes pour chaque combinaison distincte de `expressions`. Si `OFFSET` est spécifié, puis pour chaque bloc de données qui appartient à une combinaison particulière de `expressions`, Clickhouse saute `offset_value` nombre de lignes depuis le début du bloc et renvoie un maximum de `n` les lignes en conséquence. Si `offset_value` est plus grand que le nombre de lignes dans le bloc de données, ClickHouse renvoie zéro lignes du bloc. - -!!! note "Note" - `LIMIT BY` n'est pas liée à [LIMIT](limit.md). Ils peuvent tous deux être utilisés dans la même requête. - -## Exemple {#examples} - -Exemple de table: - -``` sql -CREATE TABLE limit_by(id Int, val Int) ENGINE = Memory; -INSERT INTO limit_by VALUES (1, 10), (1, 11), (1, 12), (2, 20), (2, 21); -``` - -Requête: - -``` sql -SELECT * FROM limit_by ORDER BY id, val LIMIT 2 BY id -``` - -``` text -┌─id─┬─val─┐ -│ 1 │ 10 │ -│ 1 │ 11 │ -│ 2 │ 20 │ -│ 2 │ 21 │ -└────┴─────┘ -``` - -``` sql -SELECT * FROM limit_by ORDER BY id, val LIMIT 1, 2 BY id -``` - -``` text -┌─id─┬─val─┐ -│ 1 │ 11 │ -│ 1 │ 12 │ -│ 2 │ 21 │ -└────┴─────┘ -``` - -Le `SELECT * FROM limit_by ORDER BY id, val LIMIT 2 OFFSET 1 BY id` requête renvoie le même résultat. - -La requête suivante renvoie les 5 principaux référents pour chaque `domain, device_type` paire avec un maximum de 100 lignes au total (`LIMIT n BY + LIMIT`). - -``` sql -SELECT - domainWithoutWWW(URL) AS domain, - domainWithoutWWW(REFERRER_URL) AS referrer, - device_type, - count() cnt -FROM hits -GROUP BY domain, referrer, device_type -ORDER BY cnt DESC -LIMIT 5 BY domain, device_type -LIMIT 100 -``` diff --git a/docs/fr/sql-reference/statements/select/limit.md b/docs/fr/sql-reference/statements/select/limit.md deleted file mode 100644 index 69334c32cc9..00000000000 --- a/docs/fr/sql-reference/statements/select/limit.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause LIMIT {#limit-clause} - -`LIMIT m` permet de sélectionner la première `m` lignes du résultat. - -`LIMIT n, m` permet de sélectionner le `m` lignes du résultat après avoir sauté le premier `n` rangée. Le `LIMIT m OFFSET n` la syntaxe est équivalente. - -`n` et `m` doivent être des entiers non négatifs. - -Si il n'y a pas de [ORDER BY](order-by.md) clause qui trie explicitement les résultats, le choix des lignes pour le résultat peut être arbitraire et non déterministe. diff --git a/docs/fr/sql-reference/statements/select/order-by.md b/docs/fr/sql-reference/statements/select/order-by.md deleted file mode 100644 index 2a4ef58d7ad..00000000000 --- a/docs/fr/sql-reference/statements/select/order-by.md +++ /dev/null @@ -1,72 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause ORDER BY {#select-order-by} - -Le `ORDER BY` clause contient une liste des expressions, qui peuvent être attribuées avec `DESC` (décroissant) ou `ASC` modificateur (ascendant) qui détermine la direction de tri. Si la direction n'est pas spécifié, `ASC` est supposé, donc il est généralement omis. La direction de tri s'applique à une seule expression, pas à la liste entière. Exemple: `ORDER BY Visits DESC, SearchPhrase` - -Les lignes qui ont des valeurs identiques pour la liste des expressions de tri sont sorties dans un ordre arbitraire, qui peut également être non déterministe (différent à chaque fois). -Si la clause ORDER BY est omise, l'ordre des lignes est également indéfini et peut également être non déterministe. - -## Tri des valeurs spéciales {#sorting-of-special-values} - -Il existe deux approches pour `NaN` et `NULL` ordre de tri: - -- Par défaut ou avec le `NULLS LAST` modificateur: d'abord les valeurs, puis `NaN`, puis `NULL`. -- Avec l' `NULLS FIRST` modificateur: première `NULL`, puis `NaN` puis d'autres valeurs. - -### Exemple {#example} - -Pour la table - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 2 │ -│ 1 │ nan │ -│ 2 │ 2 │ -│ 3 │ 4 │ -│ 5 │ 6 │ -│ 6 │ nan │ -│ 7 │ ᴺᵁᴸᴸ │ -│ 6 │ 7 │ -│ 8 │ 9 │ -└───┴──────┘ -``` - -Exécuter la requête `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` obtenir: - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 7 │ ᴺᵁᴸᴸ │ -│ 1 │ nan │ -│ 6 │ nan │ -│ 2 │ 2 │ -│ 2 │ 2 │ -│ 3 │ 4 │ -│ 5 │ 6 │ -│ 6 │ 7 │ -│ 8 │ 9 │ -└───┴──────┘ -``` - -Lorsque les nombres à virgule flottante sont triés, les Nan sont séparés des autres valeurs. Quel que soit l'ordre de tri, NaNs viennent à la fin. En d'autres termes, pour le Tri ascendant, ils sont placés comme s'ils étaient plus grands que tous les autres nombres, tandis que pour le Tri descendant, ils sont placés comme s'ils étaient plus petits que les autres. - -## Classement De Soutien {#collation-support} - -Pour le tri par valeurs de chaîne, vous pouvez spécifier le classement (comparaison). Exemple: `ORDER BY SearchPhrase COLLATE 'tr'` - pour le tri par mot-clé dans l'ordre croissant, en utilisant l'alphabet turc, insensible à la casse, en supposant que les chaînes sont encodées en UTF-8. COLLATE peut être spécifié ou non pour chaque expression dans L'ordre par indépendamment. Si ASC ou DESC est spécifié, COLLATE est spécifié après. Lors de L'utilisation de COLLATE, le tri est toujours insensible à la casse. - -Nous recommandons uniquement D'utiliser COLLATE pour le tri final d'un petit nombre de lignes, car le tri avec COLLATE est moins efficace que le tri normal par octets. - -## Détails De Mise En Œuvre {#implementation-details} - -Moins de RAM est utilisé si un assez petit [LIMIT](limit.md) est précisée en plus `ORDER BY`. Sinon, la quantité de mémoire dépensée est proportionnelle au volume de données à trier. Pour le traitement des requêtes distribuées, si [GROUP BY](group-by.md) est omis, le tri est partiellement effectué sur les serveurs distants et les résultats sont fusionnés Sur le serveur demandeur. Cela signifie que pour le tri distribué, le volume de données à trier peut être supérieur à la quantité de mémoire sur un seul serveur. - -S'il N'y a pas assez de RAM, il est possible d'effectuer un tri dans la mémoire externe (création de fichiers temporaires sur un disque). Utilisez le paramètre `max_bytes_before_external_sort` pour ce but. S'il est défini sur 0 (par défaut), le tri externe est désactivé. Si elle est activée, lorsque le volume de données à trier atteint le nombre spécifié d'octets, les données collectées sont triés et déposés dans un fichier temporaire. Une fois toutes les données lues, tous les fichiers triés sont fusionnés et les résultats sont générés. Les fichiers sont écrits dans le `/var/lib/clickhouse/tmp/` dans la configuration (par défaut, mais vous pouvez `tmp_path` paramètre pour modifier ce paramètre). - -L'exécution d'une requête peut utiliser plus de mémoire que `max_bytes_before_external_sort`. Pour cette raison, ce paramètre doit avoir une valeur significativement inférieure à `max_memory_usage`. Par exemple, si votre serveur dispose de 128 Go de RAM et que vous devez exécuter une seule requête, définissez `max_memory_usage` à 100 Go, et `max_bytes_before_external_sort` à 80 Go. - -Le tri externe fonctionne beaucoup moins efficacement que le tri dans la RAM. diff --git a/docs/fr/sql-reference/statements/select/prewhere.md b/docs/fr/sql-reference/statements/select/prewhere.md deleted file mode 100644 index 2c825d050f4..00000000000 --- a/docs/fr/sql-reference/statements/select/prewhere.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause PREWHERE {#prewhere-clause} - -Prewhere est une optimisation pour appliquer le filtrage plus efficacement. Il est activé par défaut, même si `PREWHERE` la clause n'est pas explicitement spécifié. Il fonctionne en déplaçant automatiquement une partie de [WHERE](where.md) condition à prewhere étape. Le rôle de `PREWHERE` la clause est seulement pour contrôler cette optimisation si vous pensez que vous savez comment le faire mieux que par défaut. - -Avec l'optimisation prewhere, au début, seules les colonnes nécessaires à l'exécution de l'expression prewhere sont lues. Ensuite, les autres colonnes sont lues qui sont nécessaires pour exécuter le reste de la requête, mais seulement les blocs où l'expression prewhere est “true” au moins pour certaines lignes. S'il y a beaucoup de blocs où prewhere expression est “false” pour toutes les lignes et prewhere a besoin de moins de colonnes que les autres parties de la requête, cela permet souvent de lire beaucoup moins de données à partir du disque pour l'exécution de la requête. - -## Contrôle Manuel De Prewhere {#controlling-prewhere-manually} - -La clause a le même sens que la `WHERE` clause. La différence est dans laquelle les données sont lues à partir de la table. Quand à commander manuellement `PREWHERE` pour les conditions de filtration qui sont utilisées par une minorité des colonnes de la requête, mais qui fournissent une filtration de données forte. Cela réduit le volume de données à lire. - -Une requête peut spécifier simultanément `PREWHERE` et `WHERE`. Dans ce cas, `PREWHERE` précéder `WHERE`. - -Si l' `optimize_move_to_prewhere` le paramètre est défini sur 0, heuristiques pour déplacer automatiquement des parties d'expressions `WHERE` de `PREWHERE` sont désactivés. - -## Limitation {#limitations} - -`PREWHERE` est uniquement pris en charge par les tables `*MergeTree` famille. diff --git a/docs/fr/sql-reference/statements/select/sample.md b/docs/fr/sql-reference/statements/select/sample.md deleted file mode 100644 index b2ddc060a19..00000000000 --- a/docs/fr/sql-reference/statements/select/sample.md +++ /dev/null @@ -1,113 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Exemple de Clause {#select-sample-clause} - -Le `SAMPLE` clause permet approchée `SELECT` le traitement de la requête. - -Lorsque l'échantillonnage de données est activé, la requête n'est pas effectuée sur toutes les données, mais uniquement sur une certaine fraction de données (échantillon). Par exemple, si vous avez besoin de calculer des statistiques pour toutes les visites, il suffit d'exécuter la requête sur le 1/10 de la fraction de toutes les visites, puis multiplier le résultat par 10. - -Le traitement approximatif des requêtes peut être utile dans les cas suivants: - -- Lorsque vous avez des exigences de synchronisation strictes (comme \<100ms), mais que vous ne pouvez pas justifier le coût des ressources matérielles supplémentaires pour y répondre. -- Lorsque vos données brutes ne sont pas précises, l'approximation ne dégrade pas sensiblement la qualité. -- Les exigences commerciales ciblent des résultats approximatifs (pour la rentabilité, ou pour commercialiser des résultats exacts aux utilisateurs premium). - -!!! note "Note" - Vous ne pouvez utiliser l'échantillonnage qu'avec les tables [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) famille, et seulement si l'expression d'échantillonnage a été spécifiée lors de la création de la table (voir [Moteur MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table)). - -Les caractéristiques de l'échantillonnage des données sont énumérées ci-dessous: - -- L'échantillonnage de données est un mécanisme déterministe. Le résultat de la même `SELECT .. SAMPLE` la requête est toujours le même. -- L'échantillonnage fonctionne de manière cohérente pour différentes tables. Pour les tables avec une seule clé d'échantillonnage, un échantillon avec le même coefficient sélectionne toujours le même sous-ensemble de données possibles. Par exemple, un exemple d'ID utilisateur prend des lignes avec le même sous-ensemble de tous les ID utilisateur possibles de différentes tables. Cela signifie que vous pouvez utiliser l'exemple dans les sous-requêtes dans la [IN](../../operators/in.md) clause. En outre, vous pouvez joindre des échantillons en utilisant le [JOIN](join.md) clause. -- L'échantillonnage permet de lire moins de données à partir d'un disque. Notez que vous devez spécifier l'échantillonnage clé correctement. Pour plus d'informations, voir [Création d'une Table MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table). - -Pour l' `SAMPLE` clause la syntaxe suivante est prise en charge: - -| SAMPLE Clause Syntax | Description | -|----------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `SAMPLE k` | Ici `k` est le nombre de 0 à 1.
La requête est exécutée sur `k` fraction des données. Exemple, `SAMPLE 0.1` exécute la requête sur 10% des données. [Lire plus](#select-sample-k) | -| `SAMPLE n` | Ici `n` est un entier suffisamment grand.
La requête est exécutée sur un échantillon d'au moins `n` lignes (mais pas significativement plus que cela). Exemple, `SAMPLE 10000000` exécute la requête sur un minimum de 10 000 000 lignes. [Lire plus](#select-sample-n) | -| `SAMPLE k OFFSET m` | Ici `k` et `m` sont les nombres de 0 à 1.
La requête est exécutée sur un échantillon de `k` fraction des données. Les données utilisées pour l'échantillon est compensée par `m` fraction. [Lire plus](#select-sample-offset) | - -## SAMPLE K {#select-sample-k} - -Ici `k` est le nombre de 0 à 1 (les notations fractionnaires et décimales sont prises en charge). Exemple, `SAMPLE 1/2` ou `SAMPLE 0.5`. - -Dans un `SAMPLE k` clause, l'échantillon est prélevé à partir de la `k` fraction des données. L'exemple est illustré ci-dessous: - -``` sql -SELECT - Title, - count() * 10 AS PageViews -FROM hits_distributed -SAMPLE 0.1 -WHERE - CounterID = 34 -GROUP BY Title -ORDER BY PageViews DESC LIMIT 1000 -``` - -Dans cet exemple, la requête est exécutée sur un échantillon de 0,1 (10%) de données. Les valeurs des fonctions d'agrégat ne sont pas corrigées automatiquement, donc pour obtenir un résultat approximatif, la valeur `count()` est multiplié manuellement par 10. - -## SAMPLE N {#select-sample-n} - -Ici `n` est un entier suffisamment grand. Exemple, `SAMPLE 10000000`. - -Dans ce cas, la requête est exécutée sur un échantillon d'au moins `n` lignes (mais pas significativement plus que cela). Exemple, `SAMPLE 10000000` exécute la requête sur un minimum de 10 000 000 lignes. - -Puisque l'unité minimale pour la lecture des données est un granule (sa taille est définie par le `index_granularity` de réglage), il est logique de définir un échantillon beaucoup plus grand que la taille du granule. - -Lors de l'utilisation de la `SAMPLE n` clause, vous ne savez pas quel pourcentage relatif de données a été traité. Donc, vous ne connaissez pas le coefficient par lequel les fonctions agrégées doivent être multipliées. L'utilisation de la `_sample_factor` colonne virtuelle pour obtenir le résultat approximatif. - -Le `_sample_factor` colonne contient des coefficients relatifs qui sont calculés dynamiquement. Cette colonne est créée automatiquement lorsque vous [créer](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) une table avec la clé d'échantillonnage spécifiée. Les exemples d'utilisation de la `_sample_factor` colonne sont indiqués ci-dessous. - -Considérons la table `visits` qui contient des statistiques sur les visites de site. Le premier exemple montre comment calculer le nombre de pages vues: - -``` sql -SELECT sum(PageViews * _sample_factor) -FROM visits -SAMPLE 10000000 -``` - -L'exemple suivant montre comment calculer le nombre total de visites: - -``` sql -SELECT sum(_sample_factor) -FROM visits -SAMPLE 10000000 -``` - -L'exemple ci-dessous montre comment calculer la durée moyenne de la session. Notez que vous n'avez pas besoin d'utiliser le coefficient relatif pour calculer les valeurs moyennes. - -``` sql -SELECT avg(Duration) -FROM visits -SAMPLE 10000000 -``` - -## SAMPLE K OFFSET M {#select-sample-offset} - -Ici `k` et `m` sont des nombres de 0 à 1. Des exemples sont présentés ci-dessous. - -**Exemple 1** - -``` sql -SAMPLE 1/10 -``` - -Dans cet exemple, l'échantillon représente 1 / 10e de toutes les données: - -`[++------------]` - -**Exemple 2** - -``` sql -SAMPLE 1/10 OFFSET 1/2 -``` - -Ici, un échantillon de 10% est prélevé à partir de la seconde moitié des données. - -`[------++------]` diff --git a/docs/fr/sql-reference/statements/select/union.md b/docs/fr/sql-reference/statements/select/union.md deleted file mode 100644 index 9ae65ebcf72..00000000000 --- a/docs/fr/sql-reference/statements/select/union.md +++ /dev/null @@ -1,35 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause UNION ALL {#union-clause} - -Vous pouvez utiliser `UNION ALL` à combiner `SELECT` requêtes en étendant leurs résultats. Exemple: - -``` sql -SELECT CounterID, 1 AS table, toInt64(count()) AS c - FROM test.hits - GROUP BY CounterID - -UNION ALL - -SELECT CounterID, 2 AS table, sum(Sign) AS c - FROM test.visits - GROUP BY CounterID - HAVING c > 0 -``` - -Les colonnes de résultat sont appariées par leur index (ordre intérieur `SELECT`). Si les noms de colonne ne correspondent pas, les noms du résultat final sont tirés de la première requête. - -La coulée de Type est effectuée pour les syndicats. Par exemple, si deux requêtes combinées ont le même champ avec non-`Nullable` et `Nullable` types d'un type compatible, la `UNION ALL` a un `Nullable` type de champ. - -Requêtes qui font partie de `UNION ALL` ne peut pas être placée entre parenthèses. [ORDER BY](order-by.md) et [LIMIT](limit.md) sont appliqués à des requêtes séparées, pas au résultat final. Si vous devez appliquer une conversion au résultat final, vous pouvez mettre toutes les requêtes avec `UNION ALL` dans une sous-requête dans la [FROM](from.md) clause. - -## Limitation {#limitations} - -Seulement `UNION ALL` est pris en charge. Régulier `UNION` (`UNION DISTINCT`) n'est pas pris en charge. Si vous avez besoin d' `UNION DISTINCT`, vous pouvez écrire `SELECT DISTINCT` à partir d'une sous-requête contenant `UNION ALL`. - -## Détails De Mise En Œuvre {#implementation-details} - -Requêtes qui font partie de `UNION ALL` peuvent être exécutées simultanément, et leurs résultats peuvent être mélangés ensemble. diff --git a/docs/fr/sql-reference/statements/select/where.md b/docs/fr/sql-reference/statements/select/where.md deleted file mode 100644 index a4d7bc5e87a..00000000000 --- a/docs/fr/sql-reference/statements/select/where.md +++ /dev/null @@ -1,15 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# Clause where {#select-where} - -`WHERE` clause permet de filtrer les données en provenance de [FROM](from.md) la clause de `SELECT`. - -Si il y a un `WHERE` , il doit contenir une expression avec la `UInt8` type. C'est généralement une expression avec comparaison et opérateurs logiques. Les lignes où cette expression est évaluée à 0 sont exclues des transformations ou des résultats ultérieurs. - -`WHERE` expression est évaluée sur la possibilité d'utiliser des index et l'élagage de partition, si le moteur de table sous-jacent le prend en charge. - -!!! note "Note" - Il y a une optimisation de filtrage appelée [prewhere](prewhere.md). diff --git a/docs/fr/sql-reference/statements/select/with.md b/docs/fr/sql-reference/statements/select/with.md deleted file mode 100644 index a42aedf460b..00000000000 --- a/docs/fr/sql-reference/statements/select/with.md +++ /dev/null @@ -1,80 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd ---- - -# AVEC la Clause {#with-clause} - -Cette section prend en charge les Expressions de Table courantes ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)), de sorte que les résultats de `WITH` la clause peut être utilisé à l'intérieur `SELECT` clause. - -## Limitation {#limitations} - -1. Les requêtes récursives ne sont pas prises en charge. -2. Lorsque la sous-requête est utilisée à l'intérieur avec section, son résultat doit être scalaire avec exactement une ligne. -3. Les résultats d'Expression ne sont pas disponibles dans les sous-requêtes. - -## Exemple {#examples} - -**Exemple 1:** Utilisation d'une expression constante comme “variable” - -``` sql -WITH '2019-08-01 15:23:00' as ts_upper_bound -SELECT * -FROM hits -WHERE - EventDate = toDate(ts_upper_bound) AND - EventTime <= ts_upper_bound -``` - -**Exemple 2:** De les expulser, somme(octets) résultat de l'expression de clause SELECT de la liste de colonnes - -``` sql -WITH sum(bytes) as s -SELECT - formatReadableSize(s), - table -FROM system.parts -GROUP BY table -ORDER BY s -``` - -**Exemple 3:** Utilisation des résultats de la sous-requête scalaire - -``` sql -/* this example would return TOP 10 of most huge tables */ -WITH - ( - SELECT sum(bytes) - FROM system.parts - WHERE active - ) AS total_disk_usage -SELECT - (sum(bytes) / total_disk_usage) * 100 AS table_disk_usage, - table -FROM system.parts -GROUP BY table -ORDER BY table_disk_usage DESC -LIMIT 10 -``` - -**Exemple 4:** Réutilisation de l'expression dans la sous-requête - -Comme solution de contournement pour la limitation actuelle de l'utilisation de l'expression dans les sous-requêtes, Vous pouvez la dupliquer. - -``` sql -WITH ['hello'] AS hello -SELECT - hello, - * -FROM -( - WITH ['hello'] AS hello - SELECT hello -) -``` - -``` text -┌─hello─────┬─hello─────┐ -│ ['hello'] │ ['hello'] │ -└───────────┴───────────┘ -``` diff --git a/docs/fr/sql-reference/statements/show.md b/docs/fr/sql-reference/statements/show.md deleted file mode 100644 index 129c6e30d1c..00000000000 --- a/docs/fr/sql-reference/statements/show.md +++ /dev/null @@ -1,169 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: SHOW ---- - -# Afficher les requêtes {#show-queries} - -## SHOW CREATE TABLE {#show-create-table} - -``` sql -SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY] [db.]table [INTO OUTFILE filename] [FORMAT format] -``` - -Renvoie un seul `String`-type ‘statement’ column, which contains a single value – the `CREATE` requête utilisée pour créer l'objet spécifié. - -## SHOW DATABASES {#show-databases} - -``` sql -SHOW DATABASES [INTO OUTFILE filename] [FORMAT format] -``` - -Imprime une liste de toutes les bases de données. -Cette requête est identique à `SELECT name FROM system.databases [INTO OUTFILE filename] [FORMAT format]`. - -## SHOW PROCESSLIST {#show-processlist} - -``` sql -SHOW PROCESSLIST [INTO OUTFILE filename] [FORMAT format] -``` - -Sorties le contenu de la [système.processus](../../operations/system-tables.md#system_tables-processes) table, qui contient une liste de requêtes en cours de traitement en ce moment, à l'exception `SHOW PROCESSLIST` requête. - -Le `SELECT * FROM system.processes` requête renvoie des données sur toutes les requêtes en cours. - -Astuce (exécuter dans la console): - -``` bash -$ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" -``` - -## SHOW TABLES {#show-tables} - -Affiche une liste de tableaux. - -``` sql -SHOW [TEMPORARY] TABLES [{FROM | IN} ] [LIKE '' | WHERE expr] [LIMIT ] [INTO OUTFILE ] [FORMAT ] -``` - -Si l' `FROM` la clause n'est pas spécifié, la requête renvoie la liste des tables de la base de données actuelle. - -Vous pouvez obtenir les mêmes résultats que l' `SHOW TABLES` requête de la façon suivante: - -``` sql -SELECT name FROM system.tables WHERE database = [AND name LIKE ] [LIMIT ] [INTO OUTFILE ] [FORMAT ] -``` - -**Exemple** - -La requête suivante sélectionne les deux premières lignes de la liste des tables `system` base de données, dont les noms contiennent `co`. - -``` sql -SHOW TABLES FROM system LIKE '%co%' LIMIT 2 -``` - -``` text -┌─name───────────────────────────┐ -│ aggregate_function_combinators │ -│ collations │ -└────────────────────────────────┘ -``` - -## SHOW DICTIONARIES {#show-dictionaries} - -Affiche une liste de [dictionnaires externes](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). - -``` sql -SHOW DICTIONARIES [FROM ] [LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] -``` - -Si l' `FROM` la clause n'est pas spécifié, la requête retourne la liste des dictionnaires de la base de données actuelle. - -Vous pouvez obtenir les mêmes résultats que l' `SHOW DICTIONARIES` requête de la façon suivante: - -``` sql -SELECT name FROM system.dictionaries WHERE database = [AND name LIKE ] [LIMIT ] [INTO OUTFILE ] [FORMAT ] -``` - -**Exemple** - -La requête suivante sélectionne les deux premières lignes de la liste des tables `system` base de données, dont les noms contiennent `reg`. - -``` sql -SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2 -``` - -``` text -┌─name─────────┐ -│ regions │ -│ region_names │ -└──────────────┘ -``` - -## SHOW GRANTS {#show-grants-statement} - -Montre les privilèges d'un utilisateur. - -### Syntaxe {#show-grants-syntax} - -``` sql -SHOW GRANTS [FOR user] -``` - -Si l'utilisateur n'est pas spécifié, la requête renvoie les privilèges de l'utilisateur actuel. - -## SHOW CREATE USER {#show-create-user-statement} - -Affiche les paramètres qui ont été utilisés [la création d'un utilisateur](create.md#create-user-statement). - -`SHOW CREATE USER` ne produit pas de mots de passe utilisateur. - -### Syntaxe {#show-create-user-syntax} - -``` sql -SHOW CREATE USER [name | CURRENT_USER] -``` - -## SHOW CREATE ROLE {#show-create-role-statement} - -Affiche les paramètres qui ont été utilisés [la création de rôle](create.md#create-role-statement) - -### Syntaxe {#show-create-role-syntax} - -``` sql -SHOW CREATE ROLE name -``` - -## SHOW CREATE ROW POLICY {#show-create-row-policy-statement} - -Affiche les paramètres qui ont été utilisés [création de stratégie de ligne](create.md#create-row-policy-statement) - -### Syntaxe {#show-create-row-policy-syntax} - -``` sql -SHOW CREATE [ROW] POLICY name ON [database.]table -``` - -## SHOW CREATE QUOTA {#show-create-quota-statement} - -Affiche les paramètres qui ont été utilisés [quota de création](create.md#create-quota-statement) - -### Syntaxe {#show-create-row-policy-syntax} - -``` sql -SHOW CREATE QUOTA [name | CURRENT] -``` - -## SHOW CREATE SETTINGS PROFILE {#show-create-settings-profile-statement} - -Affiche les paramètres qui ont été utilisés [configuration création de profil](create.md#create-settings-profile-statement) - -### Syntaxe {#show-create-row-policy-syntax} - -``` sql -SHOW CREATE [SETTINGS] PROFILE name -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/show/) diff --git a/docs/fr/sql-reference/statements/system.md b/docs/fr/sql-reference/statements/system.md deleted file mode 100644 index e8c9ed85cbc..00000000000 --- a/docs/fr/sql-reference/statements/system.md +++ /dev/null @@ -1,113 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: SYSTEM ---- - -# SYSTÈME de Requêtes {#query-language-system} - -- [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries) -- [RELOAD DICTIONARY](#query_language-system-reload-dictionary) -- [DROP DNS CACHE](#query_language-system-drop-dns-cache) -- [DROP MARK CACHE](#query_language-system-drop-mark-cache) -- [FLUSH LOGS](#query_language-system-flush_logs) -- [RELOAD CONFIG](#query_language-system-reload-config) -- [SHUTDOWN](#query_language-system-shutdown) -- [KILL](#query_language-system-kill) -- [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends) -- [FLUSH DISTRIBUTED](#query_language-system-flush-distributed) -- [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) -- [STOP MERGES](#query_language-system-stop-merges) -- [START MERGES](#query_language-system-start-merges) - -## RELOAD DICTIONARIES {#query_language-system-reload-dictionaries} - -Recharge tous les dictionnaires qui ont déjà été chargés avec succès. -Par défaut, les dictionnaires sont chargés paresseusement (voir [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load)), donc au lieu d'être chargés automatiquement au démarrage, ils sont initialisés lors du premier accès via la fonction dictGet ou sélectionnez dans les tables avec ENGINE = Dictionary . Le `SYSTEM RELOAD DICTIONARIES` query recharge ces dictionnaires (chargés). -Retourne toujours `Ok.` quel que soit le résultat de la mise à jour du dictionnaire. - -## Recharger le dictionnaire Dictionary_name {#query_language-system-reload-dictionary} - -Recharge complètement un dictionnaire `dictionary_name`, quel que soit l'état du dictionnaire (LOADED / NOT_LOADED / FAILED). -Retourne toujours `Ok.` quel que soit le résultat de la mise à jour du dictionnaire. -L'état du dictionnaire peut être vérifié en interrogeant le `system.dictionaries` table. - -``` sql -SELECT name, status FROM system.dictionaries; -``` - -## DROP DNS CACHE {#query_language-system-drop-dns-cache} - -Réinitialise le cache DNS interne de ClickHouse. Parfois (pour les anciennes versions de ClickHouse), il est nécessaire d'utiliser cette commande lors de la modification de l'infrastructure (modification de l'adresse IP d'un autre serveur ClickHouse ou du serveur utilisé par les dictionnaires). - -Pour une gestion du cache plus pratique (automatique), voir paramètres disable_internal_dns_cache, dns_cache_update_period. - -## DROP MARK CACHE {#query_language-system-drop-mark-cache} - -Réinitialise le cache de marque. Utilisé dans le développement de ClickHouse et des tests de performance. - -## FLUSH LOGS {#query_language-system-flush_logs} - -Flushes buffers of log messages to system tables (e.g. system.query_log). Allows you to not wait 7.5 seconds when debugging. - -## RELOAD CONFIG {#query_language-system-reload-config} - -Recharge la configuration de ClickHouse. Utilisé lorsque la configuration est stockée dans ZooKeeeper. - -## SHUTDOWN {#query_language-system-shutdown} - -Normalement ferme ClickHouse (comme `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`) - -## KILL {#query_language-system-kill} - -Annule le processus de ClickHouse (comme `kill -9 {$ pid_clickhouse-server}`) - -## Gestion Des Tables Distribuées {#query-language-system-distributed} - -ClickHouse peut gérer [distribué](../../engines/table-engines/special/distributed.md) table. Lorsqu'un utilisateur insère des données dans ces tables, ClickHouse crée d'abord une file d'attente des données qui doivent être envoyées aux nœuds de cluster, puis l'envoie de manière asynchrone. Vous pouvez gérer le traitement des files d'attente avec [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), et [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) requête. Vous pouvez également insérer de manière synchrone des données distribuées avec `insert_distributed_sync` paramètre. - -### STOP DISTRIBUTED SENDS {#query_language-system-stop-distributed-sends} - -Désactive la distribution de données en arrière-plan lors de l'insertion de données dans des tables distribuées. - -``` sql -SYSTEM STOP DISTRIBUTED SENDS [db.] -``` - -### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} - -Force ClickHouse à envoyer des données aux nœuds de cluster de manière synchrone. Si des nœuds ne sont pas disponibles, ClickHouse lève une exception et arrête l'exécution de la requête. Vous pouvez réessayer la requête jusqu'à ce qu'elle réussisse, ce qui se produira lorsque tous les nœuds seront de nouveau en ligne. - -``` sql -SYSTEM FLUSH DISTRIBUTED [db.] -``` - -### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} - -Active la distribution de données en arrière-plan lors de l'insertion de données dans des tables distribuées. - -``` sql -SYSTEM START DISTRIBUTED SENDS [db.] -``` - -### STOP MERGES {#query_language-system-stop-merges} - -Offre la possibilité d'arrêter les fusions d'arrière-plan pour les tables de la famille MergeTree: - -``` sql -SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] -``` - -!!! note "Note" - `DETACH / ATTACH` table va commencer les fusions d'arrière-plan pour la table même dans le cas où les fusions ont été arrêtées pour toutes les tables MergeTree auparavant. - -### START MERGES {#query_language-system-start-merges} - -Offre la possibilité de démarrer des fusions en arrière-plan pour les tables de la famille MergeTree: - -``` sql -SYSTEM START MERGES [[db.]merge_tree_family_table_name] -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/system/) diff --git a/docs/fr/sql-reference/syntax.md b/docs/fr/sql-reference/syntax.md deleted file mode 100644 index b8b24c9bbb5..00000000000 --- a/docs/fr/sql-reference/syntax.md +++ /dev/null @@ -1,187 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: Syntaxe ---- - -# Syntaxe {#syntax} - -Il existe deux types d'analyseurs dans le système: L'analyseur SQL complet (un analyseur de descente récursif) et l'analyseur de format de données (un analyseur de flux rapide). -Dans tous les cas à l'exception de la `INSERT` requête, seul L'analyseur SQL complet est utilisé. -Le `INSERT` requête utilise les deux analyseurs: - -``` sql -INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') -``` - -Le `INSERT INTO t VALUES` fragment est analysé par l'analyseur complet, et les données `(1, 'Hello, world'), (2, 'abc'), (3, 'def')` est analysé par l'analyseur de flux rapide. Vous pouvez également activer l'analyseur complet pour les données à l'aide de la [input_format_values_interpret_expressions](../operations/settings/settings.md#settings-input_format_values_interpret_expressions) paramètre. Lorsque `input_format_values_interpret_expressions = 1`, ClickHouse essaie d'abord d'analyser les valeurs avec l'analyseur de flux rapide. S'il échoue, ClickHouse essaie d'utiliser l'analyseur complet pour les données, en le traitant comme un SQL [expression](#syntax-expressions). - -Les données peuvent avoir n'importe quel format. Lorsqu'une requête est reçue, le serveur calcule pas plus que [max_query_size](../operations/settings/settings.md#settings-max_query_size) octets de la requête en RAM (par défaut, 1 Mo), et le reste est analysé en flux. -Il permet d'éviter les problèmes avec de grandes `INSERT` requête. - -Lors de l'utilisation de la `Values` format dans un `INSERT` de la requête, il peut sembler que les données sont analysées de même que les expressions dans un `SELECT` requête, mais ce n'est pas vrai. Le `Values` le format est beaucoup plus limitée. - -Le reste de cet article couvre l'analyseur complet. Pour plus d'informations sur les analyseurs de format, consultez [Format](../interfaces/formats.md) section. - -## Espace {#spaces} - -Il peut y avoir n'importe quel nombre de symboles d'espace entre les constructions syntaxiques (y compris le début et la fin d'une requête). Les symboles d'espace incluent l'espace, l'onglet, le saut de ligne, Le CR et le flux de formulaire. - -## Commentaire {#comments} - -ClickHouse prend en charge les commentaires de style SQL et de style C. -Les commentaires de style SQL commencent par `--` et continuer jusqu'à la fin de la ligne, un espace après `--` peut être omis. -C-style sont de `/*` de `*/`et peut être multiligne, les espaces ne sont pas requis non plus. - -## Mot {#syntax-keywords} - -Les mots clés sont insensibles à la casse lorsqu'ils correspondent à: - -- La norme SQL. Exemple, `SELECT`, `select` et `SeLeCt` sont toutes valides. -- Implémentation dans certains SGBD populaires (MySQL ou Postgres). Exemple, `DateTime` est le même que `datetime`. - -Si le nom du type de données est sensible à la casse peut être vérifié `system.data_type_families` table. - -Contrairement à SQL standard, tous les autres mots clés (y compris les noms de fonctions) sont **sensible à la casse**. - -Mots-clés ne sont pas réservés; ils sont traités comme tels que dans le contexte correspondant. Si vous utilisez [identificateur](#syntax-identifiers) avec le même nom que les mots-clés, placez-les entre guillemets doubles ou backticks. Par exemple, la requête `SELECT "FROM" FROM table_name` est valide si la table `table_name` a colonne avec le nom de `"FROM"`. - -## Identificateur {#syntax-identifiers} - -Les identificateurs sont: - -- Noms de Cluster, de base de données, de table, de partition et de colonne. -- Fonction. -- Types de données. -- [Expression des alias](#syntax-expression_aliases). - -Les identificateurs peuvent être cités ou non cités. Ce dernier est préféré. - -Non identificateurs doivent correspondre à l'expression régulière `^[a-zA-Z_][0-9a-zA-Z_]*$` et ne peut pas être égale à [mot](#syntax-keywords). Exemple: `x, _1, X_y__Z123_.` - -Si vous souhaitez utiliser les identifiants de la même manière que les mots-clés ou si vous souhaitez utiliser d'autres symboles dans les identifiants, citez-le en utilisant des guillemets doubles ou des backticks, par exemple, `"id"`, `` `id` ``. - -## Littéral {#literals} - -Il y a numérique, chaîne de caractères, composé, et `NULL` littéral. - -### Numérique {#numeric} - -Littéral numérique tente d'être analysé: - -- Tout d'abord, comme un nombre signé 64 bits, en utilisant le [strtoull](https://en.cppreference.com/w/cpp/string/byte/strtoul) fonction. -- En cas d'échec, en tant que nombre non signé 64 bits, [strtoll](https://en.cppreference.com/w/cpp/string/byte/strtol) fonction. -- En cas d'échec, en tant que nombre à virgule flottante [strtod](https://en.cppreference.com/w/cpp/string/byte/strtof) fonction. -- Sinon, elle renvoie une erreur. - -La valeur littérale a le plus petit type dans lequel la valeur correspond. -Par exemple, 1 est analysé comme `UInt8`, mais 256 est analysé comme `UInt16`. Pour plus d'informations, voir [Types de données](../sql-reference/data-types/index.md). - -Exemple: `1`, `18446744073709551615`, `0xDEADBEEF`, `01`, `0.1`, `1e100`, `-1e-100`, `inf`, `nan`. - -### Chaîne {#syntax-string-literal} - -Seuls les littéraux de chaîne entre guillemets simples sont pris en charge. Le clos de caractères barre oblique inverse échappé. Les séquences d'échappement suivantes ont une valeur spéciale correspondante: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\a`, `\v`, `\xHH`. Dans tous les autres cas, des séquences d'échappement au format `\c`, où `c` est un caractère, sont convertis à `c`. Cela signifie que vous pouvez utiliser les séquences `\'`et`\\`. La valeur aurez l' [Chaîne](../sql-reference/data-types/string.md) type. - -Dans les littéraux de chaîne, vous devez vous échapper d'au moins `'` et `\`. Les guillemets simples peuvent être échappés avec le guillemet simple, littéraux `'It\'s'` et `'It''s'` sont égaux. - -### Composé {#compound} - -Les tableaux sont construits avec des crochets `[1, 2, 3]`. Nuples sont construits avec des supports ronds `(1, 'Hello, world!', 2)`. -Techniquement, ce ne sont pas des littéraux, mais des expressions avec l'opérateur de création de tableau et l'opérateur de création de tuple, respectivement. -Un tableau doit être composé d'au moins un élément, et un tuple doit avoir au moins deux éléments. -Il y a un cas distinct lorsque les tuples apparaissent dans le `IN` clause de a `SELECT` requête. Les résultats de la requête peuvent inclure des tuples, mais les tuples ne peuvent pas être enregistrés dans une base de données (à l'exception des tables avec [Mémoire](../engines/table-engines/special/memory.md) moteur). - -### NULL {#null-literal} - -Indique que la valeur est manquante. - -Afin de stocker `NULL` dans un champ de table, il doit être de la [Nullable](../sql-reference/data-types/nullable.md) type. - -Selon le format de données (entrée ou sortie), `NULL` peut avoir une représentation différente. Pour plus d'informations, consultez la documentation de [formats de données](../interfaces/formats.md#formats). - -Il y a beaucoup de nuances au traitement `NULL`. Par exemple, si au moins l'un des arguments d'une opération de comparaison est `NULL` le résultat de cette opération est également `NULL`. Il en va de même pour la multiplication, l'addition et d'autres opérations. Pour plus d'informations, lisez la documentation pour chaque opération. - -Dans les requêtes, vous pouvez vérifier `NULL` à l'aide de la [IS NULL](operators/index.md#operator-is-null) et [IS NOT NULL](operators/index.md) opérateurs et les fonctions connexes `isNull` et `isNotNull`. - -## Fonction {#functions} - -Les appels de fonction sont écrits comme un identifiant avec une liste d'arguments (éventuellement vide) entre parenthèses. Contrairement à SQL standard, les crochets sont requis, même pour une liste d'arguments vide. Exemple: `now()`. -Il existe des fonctions régulières et agrégées (voir la section “Aggregate functions”). Certaines fonctions d'agrégat peut contenir deux listes d'arguments entre parenthèses. Exemple: `quantile (0.9) (x)`. Ces fonctions d'agrégation sont appelés “parametric” fonctions, et les arguments dans la première liste sont appelés “parameters”. La syntaxe des fonctions d'agrégation sans paramètres est la même que pour les fonctions régulières. - -## Opérateur {#operators} - -Les opérateurs sont convertis en leurs fonctions correspondantes lors de l'analyse des requêtes, en tenant compte de leur priorité et de leur associativité. -Par exemple, l'expression `1 + 2 * 3 + 4` est transformé à `plus(plus(1, multiply(2, 3)), 4)`. - -## Types de données et moteurs de Table de base de données {#data_types-and-database-table-engines} - -Types de données et moteurs de table dans `CREATE` les requêtes sont écrites de la même manière que les identifiants ou les fonctions. En d'autres termes, ils peuvent ou ne peuvent pas contenir une liste d'arguments entre parenthèses. Pour plus d'informations, voir les sections “Data types,” “Table engines,” et “CREATE”. - -## Expression Des Alias {#syntax-expression_aliases} - -Un alias est un nom défini par l'utilisateur pour l'expression dans une requête. - -``` sql -expr AS alias -``` - -- `AS` — The keyword for defining aliases. You can define the alias for a table name or a column name in a `SELECT` clause sans utiliser le `AS` mot. - - For example, `SELECT table_name_alias.column_name FROM table_name table_name_alias`. - - In the [CAST](sql_reference/functions/type_conversion_functions.md#type_conversion_function-cast) function, the `AS` keyword has another meaning. See the description of the function. - -- `expr` — Any expression supported by ClickHouse. - - For example, `SELECT column_name * 2 AS double FROM some_table`. - -- `alias` — Name for `expr`. Les alias doivent être conformes à la [identificateur](#syntax-identifiers) syntaxe. - - For example, `SELECT "table t".column_name FROM table_name AS "table t"`. - -### Notes sur l'Utilisation de la {#notes-on-usage} - -Les alias sont globaux pour une requête ou d'une sous-requête, vous pouvez définir un alias dans n'importe quelle partie d'une requête de toute expression. Exemple, `SELECT (1 AS n) + 2, n`. - -Les alias ne sont pas visibles dans les sous-requêtes et entre les sous-requêtes. Par exemple, lors de l'exécution de la requête `SELECT (SELECT sum(b.a) + num FROM b) - a.a AS num FROM a` Clickhouse génère l'exception `Unknown identifier: num`. - -Si un alias est défini pour les colonnes de `SELECT` la clause d'une sous-requête, ces colonnes sont visibles dans la requête externe. Exemple, `SELECT n + m FROM (SELECT 1 AS n, 2 AS m)`. - -Soyez prudent avec les Alias qui sont les mêmes que les noms de colonnes ou de tables. Considérons l'exemple suivant: - -``` sql -CREATE TABLE t -( - a Int, - b Int -) -ENGINE = TinyLog() -``` - -``` sql -SELECT - argMax(a, b), - sum(b) AS b -FROM t -``` - -``` text -Received exception from server (version 18.14.17): -Code: 184. DB::Exception: Received from localhost:9000, 127.0.0.1. DB::Exception: Aggregate function sum(b) is found inside another aggregate function in query. -``` - -Dans cet exemple, nous avons déclaré table `t` avec la colonne `b`. Ensuite, lors de la sélection des données, nous avons défini le `sum(b) AS b` alias. Comme les alias sont globaux, ClickHouse a substitué le littéral `b` dans l'expression `argMax(a, b)` avec l'expression `sum(b)`. Cette substitution a provoqué l'exception. - -## Astérisque {#asterisk} - -Dans un `SELECT` requête, un astérisque peut remplacer l'expression. Pour plus d'informations, consultez la section “SELECT”. - -## Expression {#syntax-expressions} - -Une expression est une fonction, un identifiant, un littéral, une application d'un opérateur, une expression entre parenthèses, une sous-requête ou un astérisque. Il peut également contenir un alias. -Une liste des expressions est une ou plusieurs expressions séparées par des virgules. -Les fonctions et les opérateurs, à leur tour, peuvent avoir des expressions comme arguments. - -[Article Original](https://clickhouse.tech/docs/en/sql_reference/syntax/) diff --git a/docs/fr/sql-reference/table-functions/file.md b/docs/fr/sql-reference/table-functions/file.md deleted file mode 100644 index a58821d021d..00000000000 --- a/docs/fr/sql-reference/table-functions/file.md +++ /dev/null @@ -1,121 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 37 -toc_title: fichier ---- - -# fichier {#file} - -Crée un tableau à partir d'un fichier. Cette fonction de table est similaire à [URL](url.md) et [hdfs](hdfs.md) ceux. - -``` sql -file(path, format, structure) -``` - -**Les paramètres d'entrée** - -- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Chemin d'accès à la prise en charge des fichiers suivant les globs en mode Lecture seule: `*`, `?`, `{abc,def}` et `{N..M}` où `N`, `M` — numbers, \``'abc', 'def'` — strings. -- `format` — The [format](../../interfaces/formats.md#formats) de le fichier. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - -**Valeur renvoyée** - -Une table avec la structure spécifiée pour lire ou écrire des données dans le fichier spécifié. - -**Exemple** - -Paramètre `user_files_path` et le contenu du fichier `test.csv`: - -``` bash -$ grep user_files_path /etc/clickhouse-server/config.xml - /var/lib/clickhouse/user_files/ - -$ cat /var/lib/clickhouse/user_files/test.csv - 1,2,3 - 3,2,1 - 78,43,45 -``` - -Table de`test.csv` et la sélection des deux premières lignes de ce: - -``` sql -SELECT * -FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 2 -``` - -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ -``` - -``` sql --- getting the first 10 lines of a table that contains 3 columns of UInt32 type from a CSV file -SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10 -``` - -**Globs dans le chemin** - -Plusieurs composants de chemin peuvent avoir des globs. Pour être traité, le fichier doit exister et correspondre à l'ensemble du modèle de chemin (pas seulement le suffixe ou le préfixe). - -- `*` — Substitutes any number of any characters except `/` y compris la chaîne vide. -- `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. -- `{N..M}` — Substitutes any number in range from N to M including both borders. - -Les Constructions avec `{}` sont similaires à l' [fonction de table à distance](../../sql-reference/table-functions/remote.md)). - -**Exemple** - -1. Supposons que nous ayons plusieurs fichiers avec les chemins relatifs suivants: - -- ‘some_dir/some_file_1’ -- ‘some_dir/some_file_2’ -- ‘some_dir/some_file_3’ -- ‘another_dir/some_file_1’ -- ‘another_dir/some_file_2’ -- ‘another_dir/some_file_3’ - -1. Interroger la quantité de lignes dans ces fichiers: - - - -``` sql -SELECT count(*) -FROM file('{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32') -``` - -1. Requête de la quantité de lignes dans tous les fichiers de ces deux répertoires: - - - -``` sql -SELECT count(*) -FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32') -``` - -!!! warning "Avertissement" - Si votre liste de fichiers contient des plages de nombres avec des zéros en tête, utilisez la construction avec des accolades pour chaque chiffre séparément ou utilisez `?`. - -**Exemple** - -Interroger les données des fichiers nommés `file000`, `file001`, … , `file999`: - -``` sql -SELECT count(*) -FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32') -``` - -## Les Colonnes Virtuelles {#virtual-columns} - -- `_path` — Path to the file. -- `_file` — Name of the file. - -**Voir Aussi** - -- [Les colonnes virtuelles](https://clickhouse.tech/docs/en/operations/table_engines/#table_engines-virtual_columns) - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/file/) diff --git a/docs/fr/sql-reference/table-functions/generate.md b/docs/fr/sql-reference/table-functions/generate.md deleted file mode 100644 index 1f7eeddd0e1..00000000000 --- a/docs/fr/sql-reference/table-functions/generate.md +++ /dev/null @@ -1,44 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 47 -toc_title: generateRandom ---- - -# generateRandom {#generaterandom} - -Génère des données aléatoires avec un schéma donné. -Permet de remplir des tables de test avec des données. -Prend en charge tous les types de données qui peuvent être stockés dans la table sauf `LowCardinality` et `AggregateFunction`. - -``` sql -generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]); -``` - -**Paramètre** - -- `name` — Name of corresponding column. -- `TypeName` — Type of corresponding column. -- `max_array_length` — Maximum array length for all generated arrays. Defaults to `10`. -- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`. -- `random_seed` — Specify random seed manually to produce stable results. If NULL — seed is randomly generated. - -**Valeur Renvoyée** - -Un objet de table avec le schéma demandé. - -## Exemple D'Utilisation {#usage-example} - -``` sql -SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 1, 10, 2) LIMIT 3; -``` - -``` text -┌─a────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐ -│ [77] │ -124167.6723 │ ('2061-04-17 21:59:44.573','3f72f405-ec3e-13c8-44ca-66ef335f7835') │ -│ [32,110] │ -141397.7312 │ ('1979-02-09 03:43:48.526','982486d1-5a5d-a308-e525-7bd8b80ffa73') │ -│ [68] │ -67417.0770 │ ('2080-03-12 14:17:31.269','110425e5-413f-10a6-05ba-fa6b3e929f15') │ -└──────────┴──────────────┴────────────────────────────────────────────────────────────────────┘ -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/generate/) diff --git a/docs/fr/sql-reference/table-functions/hdfs.md b/docs/fr/sql-reference/table-functions/hdfs.md deleted file mode 100644 index 51b742d8018..00000000000 --- a/docs/fr/sql-reference/table-functions/hdfs.md +++ /dev/null @@ -1,104 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 45 -toc_title: hdfs ---- - -# hdfs {#hdfs} - -Crée une table à partir de fichiers dans HDFS. Cette fonction de table est similaire à [URL](url.md) et [fichier](file.md) ceux. - -``` sql -hdfs(URI, format, structure) -``` - -**Les paramètres d'entrée** - -- `URI` — The relative URI to the file in HDFS. Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` et `{N..M}` où `N`, `M` — numbers, \``'abc', 'def'` — strings. -- `format` — The [format](../../interfaces/formats.md#formats) de le fichier. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - -**Valeur renvoyée** - -Une table avec la structure spécifiée pour lire ou écrire des données dans le fichier spécifié. - -**Exemple** - -Table de `hdfs://hdfs1:9000/test` et la sélection des deux premières lignes de ce: - -``` sql -SELECT * -FROM hdfs('hdfs://hdfs1:9000/test', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 2 -``` - -``` text -┌─column1─┬─column2─┬─column3─┐ -│ 1 │ 2 │ 3 │ -│ 3 │ 2 │ 1 │ -└─────────┴─────────┴─────────┘ -``` - -**Globs dans le chemin** - -Plusieurs composants de chemin peuvent avoir des globs. Pour être traité, le fichier doit exister et correspondre à l'ensemble du modèle de chemin (pas seulement le suffixe ou le préfixe). - -- `*` — Substitutes any number of any characters except `/` y compris la chaîne vide. -- `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. -- `{N..M}` — Substitutes any number in range from N to M including both borders. - -Les Constructions avec `{}` sont similaires à l' [fonction de table à distance](../../sql-reference/table-functions/remote.md)). - -**Exemple** - -1. Supposons que nous ayons plusieurs fichiers avec les URI suivants sur HDFS: - -- ‘hdfs://hdfs1:9000/some_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/some_dir/some_file_3’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_1’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_2’ -- ‘hdfs://hdfs1:9000/another_dir/some_file_3’ - -1. Interroger la quantité de lignes dans ces fichiers: - - - -``` sql -SELECT count(*) -FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32') -``` - -1. Requête de la quantité de lignes dans tous les fichiers de ces deux répertoires: - - - -``` sql -SELECT count(*) -FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') -``` - -!!! warning "Avertissement" - Si votre liste de fichiers contient des plages de nombres avec des zéros en tête, utilisez la construction avec des accolades pour chaque chiffre séparément ou utilisez `?`. - -**Exemple** - -Interroger les données des fichiers nommés `file000`, `file001`, … , `file999`: - -``` sql -SELECT count(*) -FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32') -``` - -## Les Colonnes Virtuelles {#virtual-columns} - -- `_path` — Path to the file. -- `_file` — Name of the file. - -**Voir Aussi** - -- [Les colonnes virtuelles](https://clickhouse.tech/docs/en/operations/table_engines/#table_engines-virtual_columns) - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/hdfs/) diff --git a/docs/fr/sql-reference/table-functions/index.md b/docs/fr/sql-reference/table-functions/index.md deleted file mode 100644 index 89a8200e385..00000000000 --- a/docs/fr/sql-reference/table-functions/index.md +++ /dev/null @@ -1,38 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Les Fonctions De Table -toc_priority: 34 -toc_title: Introduction ---- - -# Les Fonctions De Table {#table-functions} - -Les fonctions de Table sont des méthodes pour construire des tables. - -Vous pouvez utiliser les fonctions de table dans: - -- [FROM](../statements/select/from.md) la clause de la `SELECT` requête. - - The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. - -- [Créer une TABLE en tant que \< table_function ()\>](../statements/create.md#create-table-query) requête. - - It's one of the methods of creating a table. - -!!! warning "Avertissement" - Vous ne pouvez pas utiliser les fonctions de table si [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) paramètre est désactivé. - -| Fonction | Description | -|-----------------------|-------------------------------------------------------------------------------------------------------------------------------------| -| [fichier](file.md) | Crée un [Fichier](../../engines/table-engines/special/file.md)-moteur de table. | -| [fusionner](merge.md) | Crée un [Fusionner](../../engines/table-engines/special/merge.md)-moteur de table. | -| [nombre](numbers.md) | Crée une table avec une seule colonne remplie de nombres entiers. | -| [distant](remote.md) | Vous permet d'accéder à des serveurs distants sans [Distribué](../../engines/table-engines/special/distributed.md)-moteur de table. | -| [URL](url.md) | Crée un [URL](../../engines/table-engines/special/url.md)-moteur de table. | -| [mysql](mysql.md) | Crée un [MySQL](../../engines/table-engines/integrations/mysql.md)-moteur de table. | -| [jdbc](jdbc.md) | Crée un [JDBC](../../engines/table-engines/integrations/jdbc.md)-moteur de table. | -| [ODBC](odbc.md) | Crée un [ODBC](../../engines/table-engines/integrations/odbc.md)-moteur de table. | -| [hdfs](hdfs.md) | Crée un [HDFS](../../engines/table-engines/integrations/hdfs.md)-moteur de table. | - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/) diff --git a/docs/fr/sql-reference/table-functions/input.md b/docs/fr/sql-reference/table-functions/input.md deleted file mode 100644 index 21e0eacb5c1..00000000000 --- a/docs/fr/sql-reference/table-functions/input.md +++ /dev/null @@ -1,47 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 46 -toc_title: "entr\xE9e" ---- - -# entrée {#input} - -`input(structure)` - fonction de table qui permet effectivement convertir et insérer des données envoyées à la -serveur avec une structure donnée à la table avec une autre structure. - -`structure` - structure de données envoyées au serveur dans le format suivant `'column1_name column1_type, column2_name column2_type, ...'`. -Exemple, `'id UInt32, name String'`. - -Cette fonction peut être utilisée uniquement dans `INSERT SELECT` requête et une seule fois mais se comporte autrement comme une fonction de table ordinaire -(par exemple, il peut être utilisé dans la sous-requête, etc.). - -Les données peuvent être envoyées de quelque manière que ce soit comme pour ordinaire `INSERT` requête et passé dans tout disponible [format](../../interfaces/formats.md#formats) -qui doit être spécifié à la fin de la requête (contrairement à l'ordinaire `INSERT SELECT`). - -La caractéristique principale de cette fonction est que lorsque le serveur reçoit des données du client il les convertit simultanément -selon la liste des expressions dans le `SELECT` clause et insère dans la table cible. Table temporaire -avec toutes les données transférées n'est pas créé. - -**Exemple** - -- Laissez le `test` le tableau a la structure suivante `(a String, b String)` - et les données `data.csv` a une structure différente `(col1 String, col2 Date, col3 Int32)`. Requête pour insérer - les données de l' `data.csv` dans le `test` table avec conversion simultanée ressemble à ceci: - - - -``` bash -$ cat data.csv | clickhouse-client --query="INSERT INTO test SELECT lower(col1), col3 * col3 FROM input('col1 String, col2 Date, col3 Int32') FORMAT CSV"; -``` - -- Si `data.csv` contient les données de la même structure `test_structure` comme la table `test` puis ces deux requêtes sont égales: - - - -``` bash -$ cat data.csv | clickhouse-client --query="INSERT INTO test FORMAT CSV" -$ cat data.csv | clickhouse-client --query="INSERT INTO test SELECT * FROM input('test_structure') FORMAT CSV" -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/input/) diff --git a/docs/fr/sql-reference/table-functions/jdbc.md b/docs/fr/sql-reference/table-functions/jdbc.md deleted file mode 100644 index 76dea0e0930..00000000000 --- a/docs/fr/sql-reference/table-functions/jdbc.md +++ /dev/null @@ -1,29 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 43 -toc_title: jdbc ---- - -# jdbc {#table-function-jdbc} - -`jdbc(jdbc_connection_uri, schema, table)` - retourne la table qui est connectée via le pilote JDBC. - -Ce tableau fonction nécessite séparé `clickhouse-jdbc-bridge` programme en cours d'exécution. -Il prend en charge les types Nullable (basé sur DDL de la table distante qui est interrogée). - -**Exemple** - -``` sql -SELECT * FROM jdbc('jdbc:mysql://localhost:3306/?user=root&password=root', 'schema', 'table') -``` - -``` sql -SELECT * FROM jdbc('mysql://localhost:3306/?user=root&password=root', 'schema', 'table') -``` - -``` sql -SELECT * FROM jdbc('datasource://mysql-local', 'schema', 'table') -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) diff --git a/docs/fr/sql-reference/table-functions/merge.md b/docs/fr/sql-reference/table-functions/merge.md deleted file mode 100644 index 1ec264b06bd..00000000000 --- a/docs/fr/sql-reference/table-functions/merge.md +++ /dev/null @@ -1,14 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 38 -toc_title: fusionner ---- - -# fusionner {#merge} - -`merge(db_name, 'tables_regexp')` – Creates a temporary Merge table. For more information, see the section “Table engines, Merge”. - -La structure de la table est tirée de la première table rencontrée qui correspond à l'expression régulière. - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/merge/) diff --git a/docs/fr/sql-reference/table-functions/mysql.md b/docs/fr/sql-reference/table-functions/mysql.md deleted file mode 100644 index 295456914f0..00000000000 --- a/docs/fr/sql-reference/table-functions/mysql.md +++ /dev/null @@ -1,86 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 42 -toc_title: mysql ---- - -# mysql {#mysql} - -Permettre `SELECT` requêtes à effectuer sur des données stockées sur un serveur MySQL distant. - -``` sql -mysql('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']); -``` - -**Paramètre** - -- `host:port` — MySQL server address. - -- `database` — Remote database name. - -- `table` — Remote table name. - -- `user` — MySQL user. - -- `password` — User password. - -- `replace_query` — Flag that converts `INSERT INTO` les requêtes de `REPLACE INTO`. Si `replace_query=1` la requête est remplacé. - -- `on_duplicate_clause` — The `ON DUPLICATE KEY on_duplicate_clause` expression qui est ajoutée à la `INSERT` requête. - - Example: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, where `on_duplicate_clause` is `UPDATE c2 = c2 + 1`. See the MySQL documentation to find which `on_duplicate_clause` you can use with the `ON DUPLICATE KEY` clause. - - To specify `on_duplicate_clause` you need to pass `0` to the `replace_query` parameter. If you simultaneously pass `replace_query = 1` and `on_duplicate_clause`, ClickHouse generates an exception. - -Simple `WHERE` des clauses telles que `=, !=, >, >=, <, <=` sont actuellement exécutés sur le serveur MySQL. - -Le reste des conditions et le `LIMIT` les contraintes d'échantillonnage sont exécutées dans ClickHouse uniquement après la fin de la requête à MySQL. - -**Valeur Renvoyée** - -Un objet table avec les mêmes colonnes que la table MySQL d'origine. - -## Exemple D'Utilisation {#usage-example} - -Table dans MySQL: - -``` text -mysql> CREATE TABLE `test`.`test` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, - -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from test; -+------+----------+-----+----------+ -| int_id | int_nullable | float | float_nullable | -+------+----------+-----+----------+ -| 1 | NULL | 2 | NULL | -+------+----------+-----+----------+ -1 row in set (0,00 sec) -``` - -Sélection des données de ClickHouse: - -``` sql -SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123') -``` - -``` text -┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐ -│ 1 │ ᴺᵁᴸᴸ │ 2 │ ᴺᵁᴸᴸ │ -└────────┴──────────────┴───────┴────────────────┘ -``` - -## Voir Aussi {#see-also} - -- [Le ‘MySQL’ tableau moteur](../../engines/table-engines/integrations/mysql.md) -- [Utilisation de MySQL comme source de dictionnaire externe](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/mysql/) diff --git a/docs/fr/sql-reference/table-functions/numbers.md b/docs/fr/sql-reference/table-functions/numbers.md deleted file mode 100644 index 50a5ad61002..00000000000 --- a/docs/fr/sql-reference/table-functions/numbers.md +++ /dev/null @@ -1,30 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: nombre ---- - -# nombre {#numbers} - -`numbers(N)` – Returns a table with the single ‘number’ colonne (UInt64) qui contient des entiers de 0 à n-1. -`numbers(N, M)` - Retourne un tableau avec le seul ‘number’ colonne (UInt64) qui contient des entiers de N À (N + M-1). - -Similaire à la `system.numbers` table, il peut être utilisé pour tester et générer des valeurs successives, `numbers(N, M)` plus efficace que `system.numbers`. - -Les requêtes suivantes sont équivalentes: - -``` sql -SELECT * FROM numbers(10); -SELECT * FROM numbers(0, 10); -SELECT * FROM system.numbers LIMIT 10; -``` - -Exemple: - -``` sql --- Generate a sequence of dates from 2010-01-01 to 2010-12-31 -select toDate('2010-01-01') + number as d FROM numbers(365); -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/numbers/) diff --git a/docs/fr/sql-reference/table-functions/odbc.md b/docs/fr/sql-reference/table-functions/odbc.md deleted file mode 100644 index aae636a5eb2..00000000000 --- a/docs/fr/sql-reference/table-functions/odbc.md +++ /dev/null @@ -1,108 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 44 -toc_title: ODBC ---- - -# ODBC {#table-functions-odbc} - -Renvoie la table connectée via [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity). - -``` sql -odbc(connection_settings, external_database, external_table) -``` - -Paramètre: - -- `connection_settings` — Name of the section with connection settings in the `odbc.ini` fichier. -- `external_database` — Name of a database in an external DBMS. -- `external_table` — Name of a table in the `external_database`. - -Pour implémenter en toute sécurité les connexions ODBC, ClickHouse utilise un programme distinct `clickhouse-odbc-bridge`. Si le pilote ODBC est chargé directement depuis `clickhouse-server`, les problèmes de pilote peuvent planter le serveur ClickHouse. Clickhouse démarre automatiquement `clickhouse-odbc-bridge` lorsque cela est nécessaire. Le programme ODBC bridge est installé à partir du même package que `clickhouse-server`. - -Les champs avec l' `NULL` les valeurs de la table externe sont converties en valeurs par défaut pour le type de données de base. Par exemple, si un champ de table MySQL distant a `INT NULL` type il est converti en 0 (la valeur par défaut pour ClickHouse `Int32` type de données). - -## Exemple D'Utilisation {#usage-example} - -**Obtenir des données de L'installation MySQL locale via ODBC** - -Cet exemple est vérifié pour Ubuntu Linux 18.04 et MySQL server 5.7. - -Assurez-vous que unixODBC et MySQL Connector sont installés. - -Par défaut (si installé à partir de paquets), ClickHouse démarre en tant qu'utilisateur `clickhouse`. Ainsi, vous devez créer et configurer cet utilisateur dans le serveur MySQL. - -``` bash -$ sudo mysql -``` - -``` sql -mysql> CREATE USER 'clickhouse'@'localhost' IDENTIFIED BY 'clickhouse'; -mysql> GRANT ALL PRIVILEGES ON *.* TO 'clickhouse'@'clickhouse' WITH GRANT OPTION; -``` - -Puis configurez la connexion dans `/etc/odbc.ini`. - -``` bash -$ cat /etc/odbc.ini -[mysqlconn] -DRIVER = /usr/local/lib/libmyodbc5w.so -SERVER = 127.0.0.1 -PORT = 3306 -DATABASE = test -USERNAME = clickhouse -PASSWORD = clickhouse -``` - -Vous pouvez vérifier la connexion en utilisant le `isql` utilitaire de l'installation unixODBC. - -``` bash -$ isql -v mysqlconn -+-------------------------+ -| Connected! | -| | -... -``` - -Table dans MySQL: - -``` text -mysql> CREATE TABLE `test`.`test` ( - -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, - -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, - -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) - -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) - -mysql> select * from test; -+------+----------+-----+----------+ -| int_id | int_nullable | float | float_nullable | -+------+----------+-----+----------+ -| 1 | NULL | 2 | NULL | -+------+----------+-----+----------+ -1 row in set (0,00 sec) -``` - -Récupération des données de la table MySQL dans ClickHouse: - -``` sql -SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test') -``` - -``` text -┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐ -│ 1 │ 0 │ 2 │ 0 │ -└────────┴──────────────┴───────┴────────────────┘ -``` - -## Voir Aussi {#see-also} - -- [Dictionnaires externes ODBC](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) -- [Moteur de table ODBC](../../engines/table-engines/integrations/odbc.md). - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/jdbc/) diff --git a/docs/fr/sql-reference/table-functions/remote.md b/docs/fr/sql-reference/table-functions/remote.md deleted file mode 100644 index 380a9986116..00000000000 --- a/docs/fr/sql-reference/table-functions/remote.md +++ /dev/null @@ -1,85 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 40 -toc_title: distant ---- - -# à distance, remoteSecure {#remote-remotesecure} - -Vous permet d'accéder à des serveurs distants sans `Distributed` table. - -Signature: - -``` sql -remote('addresses_expr', db, table[, 'user'[, 'password']]) -remote('addresses_expr', db.table[, 'user'[, 'password']]) -remoteSecure('addresses_expr', db, table[, 'user'[, 'password']]) -remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) -``` - -`addresses_expr` – An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port` ou juste `host`. L'hôte peut être spécifié comme nom de serveur ou l'adresse IPv4 ou IPv6. Une adresse IPv6 est indiquée entre crochets. Le port est le port TCP sur le serveur distant. Si le port est omis, il utilise `tcp_port` à partir du fichier de configuration du serveur (par défaut, 9000). - -!!! important "Important" - Le port est requis pour une adresse IPv6. - -Exemple: - -``` text -example01-01-1 -example01-01-1:9000 -localhost -127.0.0.1 -[::]:9000 -[2a02:6b8:0:1111::11]:9000 -``` - -Plusieurs adresses séparées par des virgules. Dans ce cas, ClickHouse utilisera le traitement distribué, donc il enverra la requête à toutes les adresses spécifiées (comme les fragments avec des données différentes). - -Exemple: - -``` text -example01-01-1,example01-02-1 -``` - -Une partie de l'expression peut être spécifiée entre crochets. L'exemple précédent peut être écrite comme suit: - -``` text -example01-0{1,2}-1 -``` - -Les accolades peuvent contenir une plage de Nombres séparés par deux points (entiers non négatifs). Dans ce cas, la gamme est étendue à un ensemble de valeurs qui génèrent fragment d'adresses. Si le premier nombre commence par zéro, les valeurs sont formées avec le même alignement zéro. L'exemple précédent peut être écrite comme suit: - -``` text -example01-{01..02}-1 -``` - -Si vous avez plusieurs paires d'accolades, il génère le produit direct des ensembles correspondants. - -Les adresses et les parties d'adresses entre crochets peuvent être séparées par le symbole de tuyau (\|). Dans ce cas, les ensembles correspondants de adresses sont interprétés comme des répliques, et la requête sera envoyée à la première sain réplique. Cependant, les répliques sont itérées dans l'ordre actuellement défini dans [équilibrage](../../operations/settings/settings.md) paramètre. - -Exemple: - -``` text -example01-{01..02}-{1|2} -``` - -Cet exemple spécifie deux fragments qui ont chacun deux répliques. - -Le nombre d'adresses générées est limitée par une constante. En ce moment, c'est 1000 adresses. - -À l'aide de la `remote` la fonction de table est moins optimale que la création d'un `Distributed` table, car dans ce cas, la connexion au serveur est rétablie pour chaque requête. En outre, si des noms d'hôte, les noms sont résolus, et les erreurs ne sont pas comptés lors de travail avec diverses répliques. Lors du traitement d'un grand nombre de requêtes, créez toujours `Distributed` table à l'avance, et ne pas utiliser la `remote` table de fonction. - -Le `remote` table de fonction peut être utile dans les cas suivants: - -- Accès à un serveur spécifique pour la comparaison de données, le débogage et les tests. -- Requêtes entre différents clusters ClickHouse à des fins de recherche. -- Demandes distribuées peu fréquentes qui sont faites manuellement. -- Distribué demandes où l'ensemble des serveurs est redéfinie à chaque fois. - -Si l'utilisateur n'est pas spécifié, `default` est utilisée. -Si le mot de passe n'est spécifié, un mot de passe vide est utilisé. - -`remoteSecure` - la même chose que `remote` but with secured connection. Default port — [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) de config ou 9440. - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/remote/) diff --git a/docs/fr/sql-reference/table-functions/url.md b/docs/fr/sql-reference/table-functions/url.md deleted file mode 100644 index 1df5cf55526..00000000000 --- a/docs/fr/sql-reference/table-functions/url.md +++ /dev/null @@ -1,26 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 41 -toc_title: URL ---- - -# URL {#url} - -`url(URL, format, structure)` - retourne une table créée à partir du `URL` avec le -`format` et `structure`. - -URL-adresse du serveur HTTP ou HTTPS, qui peut accepter `GET` et/ou `POST` demande. - -format - [format](../../interfaces/formats.md#formats) des données. - -structure - structure de table dans `'UserID UInt64, Name String'` format. Détermine les noms et les types de colonnes. - -**Exemple** - -``` sql --- getting the first 3 lines of a table that contains columns of String and UInt32 type from HTTP-server which answers in CSV format. -SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3 -``` - -[Article Original](https://clickhouse.tech/docs/en/query_language/table_functions/url/) diff --git a/docs/fr/whats-new/changelog/2017.md b/docs/fr/whats-new/changelog/2017.md deleted file mode 120000 index d581cbbb422..00000000000 --- a/docs/fr/whats-new/changelog/2017.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/whats-new/changelog/2017.md \ No newline at end of file diff --git a/docs/fr/whats-new/changelog/2018.md b/docs/fr/whats-new/changelog/2018.md deleted file mode 120000 index 22874fcae85..00000000000 --- a/docs/fr/whats-new/changelog/2018.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/whats-new/changelog/2018.md \ No newline at end of file diff --git a/docs/fr/whats-new/changelog/2019.md b/docs/fr/whats-new/changelog/2019.md deleted file mode 120000 index 0f3f095f8a1..00000000000 --- a/docs/fr/whats-new/changelog/2019.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/whats-new/changelog/2019.md \ No newline at end of file diff --git a/docs/fr/whats-new/changelog/index.md b/docs/fr/whats-new/changelog/index.md deleted file mode 120000 index 5461b93ec8c..00000000000 --- a/docs/fr/whats-new/changelog/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/whats-new/changelog/index.md \ No newline at end of file diff --git a/docs/fr/whats-new/index.md b/docs/fr/whats-new/index.md deleted file mode 100644 index 51a77da8ef4..00000000000 --- a/docs/fr/whats-new/index.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Ce qui est Nouveau -toc_priority: 72 ---- - - diff --git a/docs/fr/whats-new/roadmap.md b/docs/fr/whats-new/roadmap.md deleted file mode 100644 index 87d64208f67..00000000000 --- a/docs/fr/whats-new/roadmap.md +++ /dev/null @@ -1,19 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 74 -toc_title: Feuille de route ---- - -# Feuille de route {#roadmap} - -## Q1 2020 {#q1-2020} - -- Contrôle d'accès par rôle - -## Q2 2020 {#q2-2020} - -- Intégration avec les services d'authentification externes -- Pools de ressources pour une répartition plus précise de la capacité du cluster entre les utilisateurs - -{## [Article Original](https://clickhouse.tech/docs/en/roadmap/) ##} diff --git a/docs/fr/whats-new/security-changelog.md b/docs/fr/whats-new/security-changelog.md deleted file mode 100644 index 6046ef96bb2..00000000000 --- a/docs/fr/whats-new/security-changelog.md +++ /dev/null @@ -1,76 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 76 -toc_title: "S\xE9curit\xE9 Changelog" ---- - -## Correction dans la version 19.14.3.3 de ClickHouse, 2019-09-10 {#fixed-in-clickhouse-release-19-14-3-3-2019-09-10} - -### CVE-2019-15024 {#cve-2019-15024} - -Аn attacker that has write access to ZooKeeper and who ican run a custom server available from the network where ClickHouse runs, can create a custom-built malicious server that will act as a ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from the malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. - -Crédits: Eldar Zaitov de L'équipe de sécurité de L'Information Yandex - -### CVE-2019-16535 {#cve-2019-16535} - -Аn OOB read, OOB write and integer underflow in decompression algorithms can be used to achieve RCE or DoS via native protocol. - -Crédits: Eldar Zaitov de L'équipe de sécurité de L'Information Yandex - -### CVE-2019-16536 {#cve-2019-16536} - -Le débordement de pile menant à DoS peut être déclenché par un client authentifié malveillant. - -Crédits: Eldar Zaitov de L'équipe de sécurité de L'Information Yandex - -## Correction de la version 19.13.6.1 de ClickHouse, 2019-09-20 {#fixed-in-clickhouse-release-19-13-6-1-2019-09-20} - -### CVE-2019-18657 {#cve-2019-18657} - -Fonction de Table `url` la vulnérabilité avait-elle permis à l'attaquant d'injecter des en-têtes HTTP arbitraires dans la requête. - -Crédit: [Nikita Tikhomirov](https://github.com/NSTikhomirov) - -## Correction dans la version ClickHouse 18.12.13, 2018-09-10 {#fixed-in-clickhouse-release-18-12-13-2018-09-10} - -### CVE-2018-14672 {#cve-2018-14672} - -Les fonctions de chargement des modèles CatBoost permettaient de parcourir les chemins et de lire des fichiers arbitraires via des messages d'erreur. - -Crédits: Andrey Krasichkov de L'équipe de sécurité de L'Information Yandex - -## Correction dans la version 18.10.3 de ClickHouse, 2018-08-13 {#fixed-in-clickhouse-release-18-10-3-2018-08-13} - -### CVE-2018-14671 {#cve-2018-14671} - -unixODBC a permis de charger des objets partagés arbitraires à partir du système de fichiers, ce qui a conduit à une vulnérabilité D'exécution de Code À Distance. - -Crédits: Andrey Krasichkov et Evgeny Sidorov de Yandex Information Security Team - -## Correction dans la version 1.1.54388 de ClickHouse, 2018-06-28 {#fixed-in-clickhouse-release-1-1-54388-2018-06-28} - -### CVE-2018-14668 {#cve-2018-14668} - -“remote” la fonction de table a permis des symboles arbitraires dans “user”, “password” et “default_database” champs qui ont conduit à des attaques de falsification de requêtes inter-protocoles. - -Crédits: Andrey Krasichkov de L'équipe de sécurité de L'Information Yandex - -## Correction dans la version 1.1.54390 de ClickHouse, 2018-07-06 {#fixed-in-clickhouse-release-1-1-54390-2018-07-06} - -### CVE-2018-14669 {#cve-2018-14669} - -Clickhouse client MySQL avait “LOAD DATA LOCAL INFILE” fonctionnalité activée permettant à une base de données MySQL malveillante de lire des fichiers arbitraires à partir du serveur clickhouse connecté. - -Crédits: Andrey Krasichkov et Evgeny Sidorov de Yandex Information Security Team - -## Correction dans la version 1.1.54131 de ClickHouse, 2017-01-10 {#fixed-in-clickhouse-release-1-1-54131-2017-01-10} - -### CVE-2018-14670 {#cve-2018-14670} - -Configuration incorrecte dans le paquet deb pourrait conduire à l'utilisation non autorisée de la base de données. - -Crédits: National Cyber Security Centre (NCSC) - -{## [Article Original](https://clickhouse.tech/docs/en/security_changelog/) ##} diff --git a/docs/tools/build.py b/docs/tools/build.py index dfb9661c326..5a1f10268ab 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -65,8 +65,6 @@ def build_for_lang(lang, args): languages = { 'en': 'English', 'zh': '中文', - 'es': 'Español', - 'fr': 'Français', 'ru': 'Русский', 'ja': '日本語' } @@ -74,8 +72,6 @@ def build_for_lang(lang, args): site_names = { 'en': 'ClickHouse %s Documentation', 'zh': 'ClickHouse文档 %s', - 'es': 'Documentación de ClickHouse %s', - 'fr': 'Documentation ClickHouse %s', 'ru': 'Документация ClickHouse %s', 'ja': 'ClickHouseドキュメント %s' } @@ -183,7 +179,7 @@ if __name__ == '__main__': website_dir = os.path.join(src_dir, 'website') arg_parser = argparse.ArgumentParser() - arg_parser.add_argument('--lang', default='en,es,fr,ru,zh,ja') + arg_parser.add_argument('--lang', default='en,ru,zh,ja') arg_parser.add_argument('--blog-lang', default='en,ru') arg_parser.add_argument('--docs-dir', default='.') arg_parser.add_argument('--theme-dir', default=website_dir) diff --git a/docs/tools/make_links.sh b/docs/tools/make_links.sh index c1194901f8f..801086178bf 100755 --- a/docs/tools/make_links.sh +++ b/docs/tools/make_links.sh @@ -8,7 +8,7 @@ BASE_DIR=$(dirname $(readlink -f $0)) function do_make_links() { set -x - langs=(en es zh fr ru ja) + langs=(en zh ru ja) src_file="$1" for lang in "${langs[@]}" do diff --git a/website/locale/es/LC_MESSAGES/messages.mo b/website/locale/es/LC_MESSAGES/messages.mo deleted file mode 100644 index 888d7a76c4ebb75778d9c20d06aa99b2f6ce2153..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 431 zcmaKo&q~8U5XLKb%F(lj5j-e5x{07=OThk#5@~EmL+{&U8YAg$+1-de`VhXJ&*DbW zdU9Z3zS)oYeeB1<{(H~Wp6f%`$DR07=Wc#;kJ#-}t)8i{7?-#pZE7Vsq}l8t8t1Tx zvUxmv^fKBgZ80%bUE-*%4CabMlnmqP?Qk;5qWRoQXE}~F3F|K4L|Ebwuz(@EM*jx< zfCuc1T`}g2n9)n!k^c2fg|V1xAx$H!(j11ZRcm*@5KXg%iX>@=DZQzXRVyk7B=ajU zC$Z)rl_W21S}hv9Q+jROYV##c7!2Th3{NB+h2WohNt{HxZ}2aeH&)V;bV0To2e9;N zFE@>-F9gjHJ6^cO&-ZpYZSbTihsG zPYw*sn|ZwVn2*DQ_dQqpu8&+Fci>N*yZOx>vD+nDJyUKmDsV~KR7!G4()mL;$zU0# zi)jAnrPL^GF*apY;J7UfW{N`?kD}S_XgW>9#llPG8BR0_>lAP*EO7`}z>r;`e+_=X z19rwP8FNHT>80wN{^iYtv6yKgO)ad_97e5G8>gR(y52%blC-layqS 1);\n" - -#: templates/common_meta.html:1 -msgid "" -"ClickHouse is a fast open-source column-oriented database management system " -"that allows generating analytical data reports in real-time using SQL queries" -msgstr "" - -#: templates/common_meta.html:6 -msgid "ClickHouse - fast open-source OLAP DBMS" -msgstr "" - -#: templates/common_meta.html:10 -msgid "ClickHouse DBMS" -msgstr "" - -#: templates/common_meta.html:32 -msgid "open-source" -msgstr "" - -#: templates/common_meta.html:32 -msgid "relational" -msgstr "" - -#: templates/common_meta.html:32 -msgid "analytics" -msgstr "" - -#: templates/common_meta.html:32 -msgid "analytical" -msgstr "" - -#: templates/common_meta.html:32 -msgid "Big Data" -msgstr "" - -#: templates/common_meta.html:32 -msgid "web-analytics" -msgstr "" - -#: templates/footer.html:8 -msgid "ClickHouse source code is published under the Apache 2.0 License." -msgstr "" - -#: templates/footer.html:8 -msgid "" -"Software is distributed on an \"AS IS\" BASIS, WITHOUT WARRANTIES OR " -"CONDITIONS OF ANY KIND, either express or implied." -msgstr "" - -#: templates/footer.html:11 -msgid "Yandex LLC" -msgstr "" - -#: templates/blog/content.html:20 templates/blog/content.html:25 -#: templates/blog/content.html:30 -msgid "Share on" -msgstr "" - -#: templates/blog/content.html:37 -msgid "Published date" -msgstr "" - -#: templates/blog/nav.html:20 -msgid "New post" -msgstr "" - -#: templates/blog/nav.html:25 -msgid "Documentation" -msgstr "" - -#: templates/docs/footer.html:3 -msgid "Rating" -msgstr "" - -#: templates/docs/footer.html:3 -msgid "votes" -msgstr "" - -#: templates/docs/footer.html:4 -msgid "Article Rating" -msgstr "" - -#: templates/docs/footer.html:4 -msgid "Was this content helpful?" -msgstr "" - -#: templates/docs/footer.html:7 -msgid "Unusable" -msgstr "" - -#: templates/docs/footer.html:7 -msgid "Poor" -msgstr "" - -#: templates/docs/footer.html:7 -msgid "Good" -msgstr "" - -#: templates/docs/footer.html:7 -msgid "Excellent" -msgstr "" - -#: templates/docs/footer.html:8 -msgid "documentation" -msgstr "" - -#: templates/docs/footer.html:15 -msgid "Built from" -msgstr "" - -#: templates/docs/footer.html:15 -msgid "published on" -msgstr "" - -#: templates/docs/footer.html:15 -msgid "modified on" -msgstr "" - -#: templates/docs/machine-translated.html:3 -msgid "Help wanted!" -msgstr "" - -#: templates/docs/machine-translated.html:4 -msgid "" -"The following content of this documentation page has been machine-" -"translated. But unlike other websites, it is not done on the fly. This " -"translated text lives on GitHub repository alongside main ClickHouse " -"codebase and waits for fellow native speakers to make it more human-readable." -msgstr "" - -#: templates/docs/machine-translated.html:4 -msgid "You can also use the original English version as a reference." -msgstr "" - -#: templates/docs/machine-translated.html:7 -msgid "Help ClickHouse documentation by editing this page" -msgstr "" - -#: templates/docs/sidebar.html:3 -msgid "Multi-page or single-page" -msgstr "" - -#: templates/docs/sidebar.html:5 -msgid "Multi-page version" -msgstr "" - -#: templates/docs/sidebar.html:8 -msgid "Single-page version" -msgstr "" - -#: templates/docs/sidebar.html:13 -msgid "Version" -msgstr "" - -#: templates/docs/sidebar.html:13 templates/docs/sidebar.html:19 -msgid "latest" -msgstr "" - -#: templates/docs/sidebar.html:36 -msgid "PDF version" -msgstr "" - -#: templates/docs/toc.html:8 -msgid "Table of Contents" -msgstr "" - -#: templates/index/community.html:4 -msgid "ClickHouse community" -msgstr "" - -#: templates/index/community.html:13 templates/index/community.html:14 -msgid "ClickHouse YouTube Channel" -msgstr "" - -#: templates/index/community.html:25 templates/index/community.html:26 -msgid "ClickHouse Official Twitter Account" -msgstr "" - -#: templates/index/community.html:36 templates/index/community.html:37 -msgid "ClickHouse at Telegram" -msgstr "" - -#: templates/index/community.html:41 -msgid "Chat with real users in " -msgstr "" - -#: templates/index/community.html:44 templates/index/community.html:116 -msgid "English" -msgstr "" - -#: templates/index/community.html:45 -msgid "or in" -msgstr "" - -#: templates/index/community.html:47 templates/index/community.html:117 -msgid "Russian" -msgstr "" - -#: templates/index/community.html:65 -msgid "Open GitHub issue to ask for help or to file a feature request" -msgstr "" - -#: templates/index/community.html:76 templates/index/community.html:77 -msgid "ClickHouse Slack Workspace" -msgstr "" - -#: templates/index/community.html:82 -msgid "Multipurpose public hangout" -msgstr "" - -#: templates/index/community.html:101 -msgid "Ask any questions" -msgstr "" - -#: templates/index/community.html:115 -msgid "ClickHouse Blog" -msgstr "" - -#: templates/index/community.html:116 -msgid "in" -msgstr "" - -#: templates/index/community.html:128 templates/index/community.html:129 -msgid "ClickHouse at Google Groups" -msgstr "" - -#: templates/index/community.html:133 -msgid "Email discussions" -msgstr "" - -#: templates/index/community.html:142 -msgid "Like ClickHouse?" -msgstr "" - -#: templates/index/community.html:143 -msgid "Help to spread the word about it via" -msgstr "" - -#: templates/index/community.html:144 -msgid "and" -msgstr "" - -#: templates/index/community.html:153 -msgid "Hosting ClickHouse Meetups" -msgstr "" - -#: templates/index/community.html:157 -msgid "" -"ClickHouse meetups are essential for strengthening community worldwide, but " -"they couldn't be possible without the help of local organizers. Please, fill " -"this form if you want to become one or want to meet ClickHouse core team for " -"any other reason." -msgstr "" - -#: templates/index/community.html:159 -msgid "ClickHouse Meetup" -msgstr "" - -#: templates/index/community.html:165 -msgid "Name" -msgstr "" - -#: templates/index/community.html:168 -msgid "Email" -msgstr "" - -#: templates/index/community.html:171 -msgid "Company" -msgstr "" - -#: templates/index/community.html:174 -msgid "City" -msgstr "" - -#: templates/index/community.html:179 -msgid "We'd like to host a public ClickHouse Meetup" -msgstr "" - -#: templates/index/community.html:185 -msgid "We'd like to invite Yandex ClickHouse team to our office" -msgstr "" - -#: templates/index/community.html:191 -msgid "We'd like to invite Yandex ClickHouse team to another event we organize" -msgstr "" - -#: templates/index/community.html:197 -msgid "We're interested in commercial consulting, support or managed service" -msgstr "" - -#: templates/index/community.html:201 -msgid "Additional comments" -msgstr "" - -#: templates/index/community.html:203 -msgid "Send" -msgstr "" - -#: templates/index/community.html:212 -msgid "" -"If you have any more thoughts or questions, feel free to contact Yandex " -"ClickHouse team directly at" -msgstr "" - -#: templates/index/community.html:213 -msgid "turn on JavaScript to see email address" -msgstr "" diff --git a/website/sitemap-index.xml b/website/sitemap-index.xml index 75fdc75973c..3fbdd99d372 100644 --- a/website/sitemap-index.xml +++ b/website/sitemap-index.xml @@ -6,21 +6,12 @@ https://clickhouse.tech/docs/zh/sitemap.xml - - https://clickhouse.tech/docs/es/sitemap.xml - - - https://clickhouse.tech/docs/fr/sitemap.xml - https://clickhouse.tech/docs/ru/sitemap.xml https://clickhouse.tech/docs/ja/sitemap.xml - - https://clickhouse.tech/docs/fa/sitemap.xml - https://clickhouse.tech/blog/en/sitemap.xml From 538be4086b07b9b95e682532dd1e4428e6ac2d79 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Apr 2021 02:59:23 +0300 Subject: [PATCH 166/266] Recursive removal is extremely dangerous. --- src/Storages/MergeTree/DataPartsExchange.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index b9ff7b41af6..3dce7aa952b 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -39,6 +39,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int S3_ERROR; extern const int INCORRECT_PART_TYPE; + extern const int LOGICAL_ERROR; } namespace DataPartsExchange @@ -543,6 +544,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( static const String TMP_PREFIX = "tmp_fetch_"; String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_; + /// We will remove directory if it's already exists. Make precautions. + if (tmp_prefix.empty() + || part_name.empty() + || std::string::npos != tmp_prefix.find_first_of("/.") + || std::string::npos != part_name.find_first_of("/.")) + throw Exception("Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.", ErrorCodes::LOGICAL_ERROR); + String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; From bbd41d64c2827932e5ca48d96935277b70429c6f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Apr 2021 02:59:47 +0300 Subject: [PATCH 167/266] Update DataPartsExchange.cpp --- src/Storages/MergeTree/DataPartsExchange.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 3dce7aa952b..862a3088f89 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -39,7 +39,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int S3_ERROR; extern const int INCORRECT_PART_TYPE; - extern const int LOGICAL_ERROR; } namespace DataPartsExchange From f7a1332574f4e225452b0c2e578d543cb841a2b0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Apr 2021 03:14:19 +0300 Subject: [PATCH 168/266] Update .gitmodules --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 3cb40a8ff14..de7250166b8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -223,4 +223,4 @@ url = https://github.com/ClickHouse-Extras/NuRaft.git [submodule "contrib/datasketches-cpp"] path = contrib/datasketches-cpp - url = https://github.com/apache/datasketches-cpp.git + url = https://github.com/ClickHouse-Extras/datasketches-cpp.git From 88d84f193435f8154bb2bfc4d9976250c11404b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 03:18:24 +0300 Subject: [PATCH 169/266] Update submodule --- contrib/datasketches-cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/datasketches-cpp b/contrib/datasketches-cpp index c1a6f8edb49..f915d35b2de 160000 --- a/contrib/datasketches-cpp +++ b/contrib/datasketches-cpp @@ -1 +1 @@ -Subproject commit c1a6f8edb49699520f248d3d02019b87429b4241 +Subproject commit f915d35b2de676683493c86c585141a1e1c83334 From 7589a014f9df04d1a29457bb4f9744291a168666 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 03:44:02 +0300 Subject: [PATCH 170/266] Better fix --- base/common/arithmeticOverflow.h | 6 ++++++ src/Functions/array/mapOp.cpp | 16 ++++++---------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/base/common/arithmeticOverflow.h b/base/common/arithmeticOverflow.h index a92fe56b9cb..c170d214636 100644 --- a/base/common/arithmeticOverflow.h +++ b/base/common/arithmeticOverflow.h @@ -25,6 +25,12 @@ namespace common return x - y; } + template + inline auto NO_SANITIZE_UNDEFINED negateIgnoreOverflow(T x) + { + return -x; + } + template inline bool addOverflow(T x, T y, T & res) { diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index ec33f48f42b..c30488cb25b 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -6,7 +6,9 @@ #include #include #include -#include "Core/ColumnWithTypeAndName.h" +#include +#include + namespace DB { @@ -120,12 +122,6 @@ private: return res; } - template - static inline auto NO_SANITIZE_UNDEFINED negate(T x) - { - return -x; - } - template ColumnPtr execute2(size_t row_count, TupleMaps & args, const DataTypeTuple & res_type) const { @@ -178,14 +174,14 @@ private: { const auto [it, inserted] = summing_map.insert({key, value}); if (!inserted) - it->second += value; + it->second = common::addIgnoreOverflow(it->second, value); } else { static_assert(op_type == OpTypes::SUBTRACT); - const auto [it, inserted] = summing_map.insert({key, first ? value : negate(value)}); + const auto [it, inserted] = summing_map.insert({key, first ? value : common::negateIgnoreOverflow(value)}); if (!inserted) - it->second -= value; + it->second = common::subIgnoreOverflow(it->second, value); } } From c65ef9bba86bf14ed156e609dd25842d53dc89fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 04:03:17 +0300 Subject: [PATCH 171/266] Add more adopters --- docs/en/introduction/adopters.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 2c1a077d112..012d86b1ef7 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -59,6 +59,7 @@ toc_title: Adopters | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | | Infovista | Networks | Analytics | — | — | [Slides in English, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) | | InnoGames | Games | Metrics, Logging | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) | +| Instabug | APM Platform | Main product | — | — | [A quote from Co-Founder](https://altinity.com/) | | Instana | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | | Integros | Platform for video services | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | | Ippon Technologies | Technology Consulting | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=205) | @@ -78,6 +79,7 @@ toc_title: Adopters | MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | | Netskope | Network Security | — | — | — | [Job advertisement, March 2021](https://www.mendeley.com/careers/job/senior-software-developer-backend-developer-1346348) | +| NIC Labs | Network Monitoring | RaTA-DNS | — | — | [Blog post, March 2021](https://niclabs.cl/ratadns/2021/03/Clickhouse) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | | OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | @@ -98,6 +100,7 @@ toc_title: Adopters | Rspamd | Antispam | Analytics | — | — | [Official Website](https://rspamd.com/doc/modules/clickhouse.html) | | RuSIEM | SIEM | Main Product | — | — | [Official Website](https://rusiem.com/en/products/architecture) | | S7 Airlines | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | +| Sber | Banking, Fintech, Retail, Cloud, Media | — | — | — | [Job advertisement, March 2021](https://career.habr.com/vacancies/1000073536) | | scireum GmbH | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | | Segment | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | | sembot.io | Shopping Ads | — | — | — | A comment on LinkedIn, 2020 | @@ -119,6 +122,7 @@ toc_title: Adopters | Tinybird | Real-time Data Products | Data processing | — | — | [Official website](https://www.tinybird.co/) | | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | +| UTMSTAT | Analytics | Main product | — | — | [Blog post, June 2020](https://vc.ru/tribuna/133956-striming-dannyh-iz-servisa-skvoznoy-analitiki-v-clickhouse) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | | VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | @@ -133,6 +137,7 @@ toc_title: Adopters | Yandex DataLens | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) | | Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | | Yandex Metrica | Web analytics | Main product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) | +| Yotascale | Cloud | Data pipeline | — | 2 bn records/day | [LinkedIn (Accomplishments)](https://www.linkedin.com/in/adilsaleem/) | | ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | From 7db8fcdebd573287fe2b8e9e6dc04cbf4c594526 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 05:37:19 +0300 Subject: [PATCH 172/266] Fix buffer overflow in TokenExtractor #19233 --- .../MergeTreeIndexAggregatorBloomFilter.h | 2 +- .../MergeTree/MergeTreeIndexBloomFilter.h | 2 +- .../MergeTreeIndexConditionBloomFilter.h | 2 +- .../MergeTree/MergeTreeIndexFullText.cpp | 74 +++++++++++++------ .../MergeTree/MergeTreeIndexFullText.h | 27 ++++--- .../MergeTreeIndexGranuleBloomFilter.h | 2 +- src/Storages/MergeTree/MergeTreeIndexMinMax.h | 6 +- src/Storages/MergeTree/MergeTreeIndexSet.h | 8 +- 8 files changed, 81 insertions(+), 42 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h index ebbe9865313..9877db8ee30 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.h @@ -6,7 +6,7 @@ namespace DB { -class MergeTreeIndexAggregatorBloomFilter : public IMergeTreeIndexAggregator +class MergeTreeIndexAggregatorBloomFilter final : public IMergeTreeIndexAggregator { public: MergeTreeIndexAggregatorBloomFilter(size_t bits_per_row_, size_t hash_functions_, const Names & columns_name_); diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h index b0d9a295bcd..1aac2c22aa0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.h @@ -8,7 +8,7 @@ namespace DB { -class MergeTreeIndexBloomFilter : public IMergeTreeIndex +class MergeTreeIndexBloomFilter final : public IMergeTreeIndex { public: MergeTreeIndexBloomFilter( diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index 34fb45c86a5..0b02e64d43c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -13,7 +13,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -class MergeTreeIndexConditionBloomFilter : public IMergeTreeIndexCondition +class MergeTreeIndexConditionBloomFilter final : public IMergeTreeIndexCondition { public: struct RPNElement diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 3e8b9cc704b..419a417c3e8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -43,15 +43,29 @@ namespace ErrorCodes /// Adds all tokens from string to bloom filter. static void stringToBloomFilter( + const String & string, TokenExtractorPtr token_extractor, BloomFilter & bloom_filter) +{ + const char * data = string.data(); + size_t size = string.size(); + + size_t cur = 0; + size_t token_start = 0; + size_t token_len = 0; + while (cur < size && token_extractor->nextInField(data, size, &cur, &token_start, &token_len)) + bloom_filter.add(data + token_start, token_len); +} + +static void columnToBloomFilter( const char * data, size_t size, TokenExtractorPtr token_extractor, BloomFilter & bloom_filter) { size_t cur = 0; size_t token_start = 0; size_t token_len = 0; - while (cur < size && token_extractor->next(data, size, &cur, &token_start, &token_len)) + while (cur < size && token_extractor->nextInColumn(data, size, &cur, &token_start, &token_len)) bloom_filter.add(data + token_start, token_len); } + /// Adds all tokens from like pattern string to bloom filter. (Because like pattern can contain `\%` and `\_`.) static void likeStringToBloomFilter( const String & data, TokenExtractorPtr token_extractor, BloomFilter & bloom_filter) @@ -61,15 +75,14 @@ static void likeStringToBloomFilter( while (cur < data.size() && token_extractor->nextLike(data, &cur, token)) bloom_filter.add(token.c_str(), token.size()); } + /// Unified condition for equals, startsWith and endsWith bool MergeTreeConditionFullText::createFunctionEqualsCondition( RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor) { out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - - const auto & str = value.get(); - stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter); + stringToBloomFilter(value.get(), token_extractor, *out.bloom_filter); return true; } @@ -143,7 +156,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, for (size_t i = 0; i < rows_read; ++i) { auto ref = column->getDataAt(*pos + i); - stringToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); + columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); } } granule->has_elems = true; @@ -367,9 +380,7 @@ bool MergeTreeConditionFullText::atomFromAST( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - - const auto & str = const_value.get(); - stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); return true; } else if (func_name == "equals") @@ -382,9 +393,7 @@ bool MergeTreeConditionFullText::atomFromAST( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - - const auto & str = const_value.get(); - likeStringToBloomFilter(str, token_extractor, *out.bloom_filter); + likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); return true; } else if (func_name == "notLike") @@ -392,9 +401,7 @@ bool MergeTreeConditionFullText::atomFromAST( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_NOT_EQUALS; out.bloom_filter = std::make_unique(params); - - const auto & str = const_value.get(); - likeStringToBloomFilter(str, token_extractor, *out.bloom_filter); + likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); return true; } else if (func_name == "hasToken") @@ -402,9 +409,7 @@ bool MergeTreeConditionFullText::atomFromAST( out.key_column = key_column_num; out.function = RPNElement::FUNCTION_EQUALS; out.bloom_filter = std::make_unique(params); - - const auto & str = const_value.get(); - stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); return true; } else if (func_name == "startsWith") @@ -431,8 +436,7 @@ bool MergeTreeConditionFullText::atomFromAST( return false; bloom_filters.back().emplace_back(params); - const auto & str = element.get(); - stringToBloomFilter(str.c_str(), str.size(), token_extractor, bloom_filters.back().back()); + stringToBloomFilter(element.get(), token_extractor, bloom_filters.back().back()); } out.set_bloom_filters = std::move(bloom_filters); return true; @@ -541,7 +545,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( { bloom_filters.back().emplace_back(params); auto ref = column->getDataAt(row); - stringToBloomFilter(ref.data, ref.size, token_extractor, bloom_filters.back().back()); + columnToBloomFilter(ref.data, ref.size, token_extractor, bloom_filters.back().back()); } } @@ -573,7 +577,7 @@ bool MergeTreeIndexFullText::mayBenefitFromIndexForIn(const ASTPtr & node) const } -bool NgramTokenExtractor::next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const +bool NgramTokenExtractor::nextInField(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const { *token_start = *pos; *token_len = 0; @@ -635,7 +639,33 @@ bool NgramTokenExtractor::nextLike(const String & str, size_t * pos, String & to return false; } -bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const + +bool SplitTokenExtractor::nextInField(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const +{ + *token_start = *pos; + *token_len = 0; + + while (*pos < len) + { + if (isASCII(data[*pos]) && !isAlphaNumericASCII(data[*pos])) + { + /// Finish current token if any + if (*token_len > 0) + return true; + *token_start = ++*pos; + } + else + { + /// Note that UTF-8 sequence is completely consisted of non-ASCII bytes. + ++*pos; + ++*token_len; + } + } + + return *token_len > 0; +} + +bool SplitTokenExtractor::nextInColumn(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const { *token_start = *pos; *token_len = 0; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index c3c1ff8de8b..d861751c7df 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -14,10 +14,18 @@ namespace DB struct ITokenExtractor { virtual ~ITokenExtractor() = default; + /// Fast inplace implementation for regular use. /// Gets string (data ptr and len) and start position for extracting next token (state of extractor). /// Returns false if parsing is finished, otherwise returns true. - virtual bool next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const = 0; + virtual bool nextInField(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const = 0; + + /// Optimized version that can assume at least 15 padding bytes after data + len (as our Columns provide). + virtual bool nextInColumn(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const + { + return nextInField(data, len, pos, token_start, token_len); + } + /// Special implementation for creating bloom filter for LIKE function. /// It skips unescaped `%` and `_` and supports escaping symbols, but it is less lightweight. virtual bool nextLike(const String & str, size_t * pos, String & out) const = 0; @@ -27,7 +35,7 @@ struct ITokenExtractor using TokenExtractorPtr = const ITokenExtractor *; -struct MergeTreeIndexGranuleFullText : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleFullText final : public IMergeTreeIndexGranule { explicit MergeTreeIndexGranuleFullText( const String & index_name_, @@ -50,7 +58,7 @@ struct MergeTreeIndexGranuleFullText : public IMergeTreeIndexGranule using MergeTreeIndexGranuleFullTextPtr = std::shared_ptr; -struct MergeTreeIndexAggregatorFullText : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorFullText final : IMergeTreeIndexAggregator { explicit MergeTreeIndexAggregatorFullText( const Names & index_columns_, @@ -74,7 +82,7 @@ struct MergeTreeIndexAggregatorFullText : IMergeTreeIndexAggregator }; -class MergeTreeConditionFullText : public IMergeTreeIndexCondition +class MergeTreeConditionFullText final : public IMergeTreeIndexCondition { public: MergeTreeConditionFullText( @@ -156,13 +164,13 @@ private: /// Parser extracting all ngrams from string. -struct NgramTokenExtractor : public ITokenExtractor +struct NgramTokenExtractor final : public ITokenExtractor { NgramTokenExtractor(size_t n_) : n(n_) {} static String getName() { return "ngrambf_v1"; } - bool next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const override; + bool nextInField(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const override; bool nextLike(const String & str, size_t * pos, String & token) const override; bool supportLike() const override { return true; } @@ -171,18 +179,19 @@ struct NgramTokenExtractor : public ITokenExtractor }; /// Parser extracting tokens (sequences of numbers and ascii letters). -struct SplitTokenExtractor : public ITokenExtractor +struct SplitTokenExtractor final : public ITokenExtractor { static String getName() { return "tokenbf_v1"; } - bool next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const override; + bool nextInField(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const override; + bool nextInColumn(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const override; bool nextLike(const String & str, size_t * pos, String & token) const override; bool supportLike() const override { return true; } }; -class MergeTreeIndexFullText : public IMergeTreeIndex +class MergeTreeIndexFullText final : public IMergeTreeIndex { public: MergeTreeIndexFullText( diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h index 54e2c105db8..cdd4b92f80c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h @@ -6,7 +6,7 @@ namespace DB { -class MergeTreeIndexGranuleBloomFilter : public IMergeTreeIndexGranule +class MergeTreeIndexGranuleBloomFilter final : public IMergeTreeIndexGranule { public: MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_); diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.h b/src/Storages/MergeTree/MergeTreeIndexMinMax.h index 3956b1d9f9a..8d782d9a7dc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.h +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.h @@ -10,7 +10,7 @@ namespace DB { -struct MergeTreeIndexGranuleMinMax : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleMinMax final : public IMergeTreeIndexGranule { MergeTreeIndexGranuleMinMax(const String & index_name_, const Block & index_sample_block_); MergeTreeIndexGranuleMinMax( @@ -31,7 +31,7 @@ struct MergeTreeIndexGranuleMinMax : public IMergeTreeIndexGranule }; -struct MergeTreeIndexAggregatorMinMax : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorMinMax final : IMergeTreeIndexAggregator { MergeTreeIndexAggregatorMinMax(const String & index_name_, const Block & index_sample_block); ~MergeTreeIndexAggregatorMinMax() override = default; @@ -46,7 +46,7 @@ struct MergeTreeIndexAggregatorMinMax : IMergeTreeIndexAggregator }; -class MergeTreeIndexConditionMinMax : public IMergeTreeIndexCondition +class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition { public: MergeTreeIndexConditionMinMax( diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.h b/src/Storages/MergeTree/MergeTreeIndexSet.h index d84991f5e85..90389264d53 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.h +++ b/src/Storages/MergeTree/MergeTreeIndexSet.h @@ -14,7 +14,7 @@ namespace DB class MergeTreeIndexSet; -struct MergeTreeIndexGranuleSet : public IMergeTreeIndexGranule +struct MergeTreeIndexGranuleSet final : public IMergeTreeIndexGranule { explicit MergeTreeIndexGranuleSet( const String & index_name_, @@ -42,7 +42,7 @@ struct MergeTreeIndexGranuleSet : public IMergeTreeIndexGranule }; -struct MergeTreeIndexAggregatorSet : IMergeTreeIndexAggregator +struct MergeTreeIndexAggregatorSet final : IMergeTreeIndexAggregator { explicit MergeTreeIndexAggregatorSet( const String & index_name_, @@ -79,7 +79,7 @@ private: }; -class MergeTreeIndexConditionSet : public IMergeTreeIndexCondition +class MergeTreeIndexConditionSet final : public IMergeTreeIndexCondition { public: MergeTreeIndexConditionSet( @@ -113,7 +113,7 @@ private: }; -class MergeTreeIndexSet : public IMergeTreeIndex +class MergeTreeIndexSet final : public IMergeTreeIndex { public: MergeTreeIndexSet( From 269ccabc0cba8f5349958c23ecda47f067735bef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 05:39:31 +0300 Subject: [PATCH 173/266] Add test --- .../01781_token_extractor_buffer_overflow.reference | 0 .../01781_token_extractor_buffer_overflow.sql | 10 ++++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference create mode 100644 tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql new file mode 100644 index 00000000000..737ac7226a5 --- /dev/null +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql @@ -0,0 +1,10 @@ +SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20, max_rows_to_read = 16; + +DROP TABLE IF EXISTS bloom_filter; +CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8; +INSERT INTO bloom_filter SELECT number, 'yyy,uuu' FROM numbers(1024); + +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); +SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abcabcabcabcabcabcabcab\0'); + +DROP TABLE bloom_filter; From 14f2cf9b801c279bf420772316d3330778d85964 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 06:17:23 +0300 Subject: [PATCH 174/266] Add memory accounting in parallel formatting --- .../Impl/ParallelFormattingOutputFormat.cpp | 8 ++++++-- .../Formats/Impl/ParallelFormattingOutputFormat.h | 14 ++++++++++---- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index 0ebca3661b4..ce7dd1abd51 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -80,9 +80,11 @@ namespace DB } - void ParallelFormattingOutputFormat::collectorThreadFunction() + void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupStatusPtr & thread_group) { setThreadName("Collector"); + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); try { @@ -135,9 +137,11 @@ namespace DB } - void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number) + void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, const ThreadGroupStatusPtr & thread_group) { setThreadName("Formatter"); + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); try { diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 584aa364d27..8b9e8293c69 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -76,7 +76,10 @@ public: /// Just heuristic. We need one thread for collecting, one thread for receiving chunks /// and n threads for formatting. processing_units.resize(params.max_threads_for_parallel_formatting + 2); - collector_thread = ThreadFromGlobalPool([&] { collectorThreadFunction(); }); + collector_thread = ThreadFromGlobalPool([thread_group = CurrentThread::getGroup(), this] + { + collectorThreadFunction(thread_group); + }); LOG_TRACE(&Poco::Logger::get("ParallelFormattingOutputFormat"), "Parallel formatting is being used"); } @@ -200,14 +203,17 @@ private: void scheduleFormatterThreadForUnitWithNumber(size_t ticket_number) { - pool.scheduleOrThrowOnError([this, ticket_number] { formatterThreadFunction(ticket_number); }); + pool.scheduleOrThrowOnError([this, thread_group = CurrentThread::getGroup(), ticket_number] + { + formatterThreadFunction(ticket_number, thread_group); + }); } /// Collects all temporary buffers into main WriteBuffer. - void collectorThreadFunction(); + void collectorThreadFunction(const ThreadGroupStatusPtr & thread_group); /// This function is executed in ThreadPool and the only purpose of it is to format one Chunk into a continuous buffer in memory. - void formatterThreadFunction(size_t current_unit_number); + void formatterThreadFunction(size_t current_unit_number, const ThreadGroupStatusPtr & thread_group); }; } From da769ef6c374869e5aa7bbe4842a3aa060594d98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 06:22:05 +0300 Subject: [PATCH 175/266] Add a test --- .../0_stateless/01783_parallel_formatting_memory.reference | 1 + .../0_stateless/01783_parallel_formatting_memory.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01783_parallel_formatting_memory.reference create mode 100755 tests/queries/0_stateless/01783_parallel_formatting_memory.sh diff --git a/tests/queries/0_stateless/01783_parallel_formatting_memory.reference b/tests/queries/0_stateless/01783_parallel_formatting_memory.reference new file mode 100644 index 00000000000..c5cdc5cf0bb --- /dev/null +++ b/tests/queries/0_stateless/01783_parallel_formatting_memory.reference @@ -0,0 +1 @@ +Code: 241 diff --git a/tests/queries/0_stateless/01783_parallel_formatting_memory.sh b/tests/queries/0_stateless/01783_parallel_formatting_memory.sh new file mode 100755 index 00000000000..0b8cb0bc6be --- /dev/null +++ b/tests/queries/0_stateless/01783_parallel_formatting_memory.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&max_memory_usage=1G" -d "SELECT range(65535) FROM system.one ARRAY JOIN range(65536) AS number" | grep -oF 'Code: 241' From b0539eb28f44ecff1079696549362e2801a6ccb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 06:28:25 +0300 Subject: [PATCH 176/266] Move method --- src/DataStreams/NativeBlockOutputStream.cpp | 2 +- src/DataStreams/NativeBlockOutputStream.h | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index da68376201f..2a016c9a0c8 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -41,7 +41,7 @@ void NativeBlockOutputStream::flush() } -void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) diff --git a/src/DataStreams/NativeBlockOutputStream.h b/src/DataStreams/NativeBlockOutputStream.h index 64ccd267634..c47d7b2f1c3 100644 --- a/src/DataStreams/NativeBlockOutputStream.h +++ b/src/DataStreams/NativeBlockOutputStream.h @@ -30,8 +30,6 @@ public: void write(const Block & block) override; void flush() override; - static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit); - String getContentType() const override { return "application/octet-stream"; } private: From 9d65d83c8320104438bfc6d6413708de1c16ee03 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 07:12:07 +0300 Subject: [PATCH 177/266] Better exception message in client in case of exception while writing blocks --- src/Server/TCPHandler.cpp | 53 +++++++++++++++++++++++++++++---------- 1 file changed, 40 insertions(+), 13 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ae9358c6159..efda9bbfec3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1409,22 +1409,49 @@ void TCPHandler::sendData(const Block & block) { initBlockOutput(block); - writeVarUInt(Protocol::Server::Data, *out); - /// Send external table name (empty name is the main table) - writeStringBinary("", *out); + auto prev_bytes_written_out = out->count(); + auto prev_bytes_written_compressed_out = state.maybe_compressed_out->count(); - /// For testing hedged requests - const Settings & settings = query_context->getSettingsRef(); - if (block.rows() > 0 && settings.sleep_in_send_data_ms.totalMilliseconds()) + try { - out->next(); - std::chrono::milliseconds ms(settings.sleep_in_send_data_ms.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } + writeVarUInt(Protocol::Server::Data, *out); + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); - state.block_out->write(block); - state.maybe_compressed_out->next(); - out->next(); + /// For testing hedged requests + const Settings & settings = query_context->getSettingsRef(); + if (block.rows() > 0 && settings.sleep_in_send_data_ms.totalMilliseconds()) + { + out->next(); + std::chrono::milliseconds ms(settings.sleep_in_send_data_ms.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + + state.block_out->write(block); + state.maybe_compressed_out->next(); + out->next(); + } + catch (...) + { + /// In case of unsuccessful write, if the buffer with written data was not flushed, + /// we will rollback write to avoid breaking the protocol. + /// (otherwise the client will not be able to receive exception after unfinished data + /// as it will expect the continuation of the data). + /// It looks like hangs on client side or a message like "Data compressed with different methods". + + if (state.compression == Protocol::Compression::Enable) + { + auto extra_bytes_written_compressed = state.maybe_compressed_out->count() - prev_bytes_written_compressed_out; + if (state.maybe_compressed_out->offset() >= extra_bytes_written_compressed) + state.maybe_compressed_out->position() -= extra_bytes_written_compressed; + } + + auto extra_bytes_written_out = out->count() - prev_bytes_written_out; + if (out->offset() >= extra_bytes_written_out) + out->position() -= extra_bytes_written_out; + + throw; + } } From 507dae3bdcc02f903af563bf5dc4d4ac6f973042 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 07:12:12 +0300 Subject: [PATCH 178/266] Add test --- .../0_stateless/01784_parallel_formatting_memory.reference | 0 tests/queries/0_stateless/01784_parallel_formatting_memory.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01784_parallel_formatting_memory.reference create mode 100644 tests/queries/0_stateless/01784_parallel_formatting_memory.sql diff --git a/tests/queries/0_stateless/01784_parallel_formatting_memory.reference b/tests/queries/0_stateless/01784_parallel_formatting_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01784_parallel_formatting_memory.sql b/tests/queries/0_stateless/01784_parallel_formatting_memory.sql new file mode 100644 index 00000000000..35dc063f895 --- /dev/null +++ b/tests/queries/0_stateless/01784_parallel_formatting_memory.sql @@ -0,0 +1,2 @@ +SET max_memory_usage = '1G'; +SELECT range(65535) FROM system.one ARRAY JOIN range(65536) AS number; -- { serverError 241 } From f9c9086ab21ad687086f3019f904456b31c73fb1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 07:13:43 +0300 Subject: [PATCH 179/266] More tests --- .../01785_parallel_formatting_memory.reference | 2 ++ .../0_stateless/01785_parallel_formatting_memory.sh | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/01785_parallel_formatting_memory.reference create mode 100755 tests/queries/0_stateless/01785_parallel_formatting_memory.sh diff --git a/tests/queries/0_stateless/01785_parallel_formatting_memory.reference b/tests/queries/0_stateless/01785_parallel_formatting_memory.reference new file mode 100644 index 00000000000..0ec7fc54b01 --- /dev/null +++ b/tests/queries/0_stateless/01785_parallel_formatting_memory.reference @@ -0,0 +1,2 @@ +Code: 241 +Code: 241 diff --git a/tests/queries/0_stateless/01785_parallel_formatting_memory.sh b/tests/queries/0_stateless/01785_parallel_formatting_memory.sh new file mode 100755 index 00000000000..6d081c61fd3 --- /dev/null +++ b/tests/queries/0_stateless/01785_parallel_formatting_memory.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --compress 0 --max_memory_usage 1G --query "SELECT range(65535) FROM system.one ARRAY JOIN range(65536) AS number" 2>&1 | grep -oF 'Code: 241' | head -n1 +$CLICKHOUSE_CLIENT --compress 1 --max_memory_usage 1G --query "SELECT range(65535) FROM system.one ARRAY JOIN range(65536) AS number" 2>&1 | grep -oF 'Code: 241' | head -n1 From d6b81464ddb232d29d32ca90fb7b94939953af8b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Apr 2021 07:16:23 +0300 Subject: [PATCH 180/266] Update 01781_token_extractor_buffer_overflow.sql --- .../0_stateless/01781_token_extractor_buffer_overflow.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql index 737ac7226a5..00be1f46aba 100644 --- a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql @@ -1,4 +1,4 @@ -SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20, max_rows_to_read = 16; +SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20; DROP TABLE IF EXISTS bloom_filter; CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8; From cd7e48c64bf9d5bc8e6fba07375b2ffecab82db3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Apr 2021 08:51:12 +0300 Subject: [PATCH 181/266] Update 01781_token_extractor_buffer_overflow.reference --- .../0_stateless/01781_token_extractor_buffer_overflow.reference | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference index e69de29bb2d..aeba736d914 100644 --- a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference @@ -0,0 +1,2 @@ +0 +0 From 4daf011e0b3a291aa7ebc7c42d8a9b767d9f5ce2 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 8 Mar 2021 17:15:44 +0800 Subject: [PATCH 182/266] add aliases to visitParam/visitParamExtract* functions --- src/Functions/visitParamExtractBool.cpp | 3 +++ src/Functions/visitParamExtractFloat.cpp | 3 +++ src/Functions/visitParamExtractInt.cpp | 3 +++ src/Functions/visitParamExtractRaw.cpp | 3 +++ src/Functions/visitParamExtractString.cpp | 3 +++ src/Functions/visitParamExtractUInt.cpp | 4 ++++ src/Functions/visitParamHas.cpp | 3 +++ 7 files changed, 22 insertions(+) diff --git a/src/Functions/visitParamExtractBool.cpp b/src/Functions/visitParamExtractBool.cpp index 7f989ccbb9e..918826abba3 100644 --- a/src/Functions/visitParamExtractBool.cpp +++ b/src/Functions/visitParamExtractBool.cpp @@ -19,10 +19,13 @@ struct ExtractBool struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; }; using FunctionVisitParamExtractBool = FunctionsStringSearch, NameVisitParamExtractBool>; +struct NameJSONSExtractBool { static constexpr auto name = "JSONSExtractBool"; }; +using FunctionJSONSExtractBool = FunctionsStringSearch, NameJSONSExtractBool>; void registerFunctionVisitParamExtractBool(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractFloat.cpp b/src/Functions/visitParamExtractFloat.cpp index b02b0209daf..7533299b9e4 100644 --- a/src/Functions/visitParamExtractFloat.cpp +++ b/src/Functions/visitParamExtractFloat.cpp @@ -9,10 +9,13 @@ namespace DB struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; }; using FunctionVisitParamExtractFloat = FunctionsStringSearch>, NameVisitParamExtractFloat>; +struct NameJSONSExtractFloat { static constexpr auto name = "JSONSExtractFloat"; }; +using FunctionJSONSExtractFloat = FunctionsStringSearch>, NameJSONSExtractFloat>; void registerFunctionVisitParamExtractFloat(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractInt.cpp b/src/Functions/visitParamExtractInt.cpp index f3f30f566e6..692511c306a 100644 --- a/src/Functions/visitParamExtractInt.cpp +++ b/src/Functions/visitParamExtractInt.cpp @@ -9,10 +9,13 @@ namespace DB struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; }; using FunctionVisitParamExtractInt = FunctionsStringSearch>, NameVisitParamExtractInt>; +struct NameJSONSExtractInt { static constexpr auto name = "JSONSExtractInt"; }; +using FunctionJSONSExtractInt = FunctionsStringSearch>, NameJSONSExtractInt>; void registerFunctionVisitParamExtractInt(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index add882f003f..47304eb8273 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -59,10 +59,13 @@ struct ExtractRaw struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; }; using FunctionVisitParamExtractRaw = FunctionsStringSearchToString, NameVisitParamExtractRaw>; +struct NameJSONSExtractRaw { static constexpr auto name = "JSONSExtractRaw"; }; +using FunctionJSONSExtractRaw = FunctionsStringSearchToString, NameJSONSExtractRaw>; void registerFunctionVisitParamExtractRaw(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractString.cpp b/src/Functions/visitParamExtractString.cpp index b633a59807e..a7df64e6be4 100644 --- a/src/Functions/visitParamExtractString.cpp +++ b/src/Functions/visitParamExtractString.cpp @@ -20,10 +20,13 @@ struct ExtractString struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; }; using FunctionVisitParamExtractString = FunctionsStringSearchToString, NameVisitParamExtractString>; +struct NameJSONSExtractString { static constexpr auto name = "JSONSExtractString"; }; +using FunctionJSONSExtractString = FunctionsStringSearchToString, NameJSONSExtractString>; void registerFunctionVisitParamExtractString(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractUInt.cpp b/src/Functions/visitParamExtractUInt.cpp index 5e70eed8253..bf7bf211b7d 100644 --- a/src/Functions/visitParamExtractUInt.cpp +++ b/src/Functions/visitParamExtractUInt.cpp @@ -9,10 +9,14 @@ namespace DB struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; }; using FunctionVisitParamExtractUInt = FunctionsStringSearch>, NameVisitParamExtractUInt>; +struct NameJSONSExtractUInt { static constexpr auto name = "JSONSExtractUInt"; }; +using FunctionJSONSExtractUInt = FunctionsStringSearch>, NameJSONSExtractUInt>; + void registerFunctionVisitParamExtractUInt(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamHas.cpp b/src/Functions/visitParamHas.cpp index 5fbedfb4995..a6dc2df41ba 100644 --- a/src/Functions/visitParamHas.cpp +++ b/src/Functions/visitParamHas.cpp @@ -19,10 +19,13 @@ struct HasParam struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; }; using FunctionVisitParamHas = FunctionsStringSearch, NameVisitParamHas>; +struct NameJSONSHas { static constexpr auto name = "JSONSHas"; }; +using FunctionJSONSHas = FunctionsStringSearch, NameJSONSHas>; void registerFunctionVisitParamHas(FunctionFactory & factory) { factory.registerFunction(); + factory.registerFunction(); } } From bbb8a4439953d3d8d0e7c421872016067323d914 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Mar 2021 12:22:54 +0000 Subject: [PATCH 183/266] Add test --- .../00539_functions_for_working_with_json.reference | 7 +++++++ .../0_stateless/00539_functions_for_working_with_json.sql | 8 ++++++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference index c0399f8ab2e..4d3527722a1 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference @@ -13,3 +13,10 @@ test"string "[" ["]", "2", "3"] {"nested" : [1,2,3]} +-1 +0 +0 +-1 +1 +test_string +test"string diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql index 514b5f2e5ea..f6840947ae7 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql @@ -15,3 +15,11 @@ SELECT visitParamExtractRaw('{"myparam": "{"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "["}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": ["]", "2", "3"], "other":123}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": {"nested" : [1,2,3]}, "other":123}', 'myparam'); + +SELECT JSONExtractInt('{"myparam":-1}', 'myparam'); +SELECT JSONExtractUInt('{"myparam":-1}', 'myparam'); +SELECT JSONExtractFloat('{"myparam":null}', 'myparam'); +SELECT JSONExtractFloat('{"myparam":-1}', 'myparam'); +SELECT JSONExtractBool('{"myparam":true}', 'myparam'); +SELECT JSONExtractString('{"myparam":"test_string"}', 'myparam'); +SELECT JSONExtractString('{"myparam":"test\\"string"}', 'myparam'); From 799a1f177eb857ac7004dabd37e6c5a58d0cd496 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 30 Mar 2021 16:25:56 +0000 Subject: [PATCH 184/266] Rename to SimpleJSON* --- src/Functions/visitParamExtractBool.cpp | 6 +++--- src/Functions/visitParamExtractFloat.cpp | 6 +++--- src/Functions/visitParamExtractInt.cpp | 6 +++--- src/Functions/visitParamExtractRaw.cpp | 6 +++--- src/Functions/visitParamExtractString.cpp | 6 +++--- src/Functions/visitParamExtractUInt.cpp | 6 +++--- src/Functions/visitParamHas.cpp | 6 +++--- .../00539_functions_for_working_with_json.sql | 14 +++++++------- 8 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/Functions/visitParamExtractBool.cpp b/src/Functions/visitParamExtractBool.cpp index 918826abba3..a1b6c83180b 100644 --- a/src/Functions/visitParamExtractBool.cpp +++ b/src/Functions/visitParamExtractBool.cpp @@ -19,13 +19,13 @@ struct ExtractBool struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; }; using FunctionVisitParamExtractBool = FunctionsStringSearch, NameVisitParamExtractBool>; -struct NameJSONSExtractBool { static constexpr auto name = "JSONSExtractBool"; }; -using FunctionJSONSExtractBool = FunctionsStringSearch, NameJSONSExtractBool>; +struct NameSimpleJSONExtractBool { static constexpr auto name = "SimpleJSONExtractBool"; }; +using FunctionSimpleJSONExtractBool = FunctionsStringSearch, NameSimpleJSONExtractBool>; void registerFunctionVisitParamExtractBool(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractFloat.cpp b/src/Functions/visitParamExtractFloat.cpp index 7533299b9e4..0ddfca95f1d 100644 --- a/src/Functions/visitParamExtractFloat.cpp +++ b/src/Functions/visitParamExtractFloat.cpp @@ -9,13 +9,13 @@ namespace DB struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; }; using FunctionVisitParamExtractFloat = FunctionsStringSearch>, NameVisitParamExtractFloat>; -struct NameJSONSExtractFloat { static constexpr auto name = "JSONSExtractFloat"; }; -using FunctionJSONSExtractFloat = FunctionsStringSearch>, NameJSONSExtractFloat>; +struct NameSimpleJSONExtractFloat { static constexpr auto name = "SimpleJSONExtractFloat"; }; +using FunctionSimpleJSONExtractFloat = FunctionsStringSearch>, NameSimpleJSONExtractFloat>; void registerFunctionVisitParamExtractFloat(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractInt.cpp b/src/Functions/visitParamExtractInt.cpp index 692511c306a..094e3d966af 100644 --- a/src/Functions/visitParamExtractInt.cpp +++ b/src/Functions/visitParamExtractInt.cpp @@ -9,13 +9,13 @@ namespace DB struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; }; using FunctionVisitParamExtractInt = FunctionsStringSearch>, NameVisitParamExtractInt>; -struct NameJSONSExtractInt { static constexpr auto name = "JSONSExtractInt"; }; -using FunctionJSONSExtractInt = FunctionsStringSearch>, NameJSONSExtractInt>; +struct NameSimpleJSONExtractInt { static constexpr auto name = "SimpleJSONExtractInt"; }; +using FunctionSimpleJSONExtractInt = FunctionsStringSearch>, NameSimpleJSONExtractInt>; void registerFunctionVisitParamExtractInt(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index 47304eb8273..f1f73be2847 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -59,13 +59,13 @@ struct ExtractRaw struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; }; using FunctionVisitParamExtractRaw = FunctionsStringSearchToString, NameVisitParamExtractRaw>; -struct NameJSONSExtractRaw { static constexpr auto name = "JSONSExtractRaw"; }; -using FunctionJSONSExtractRaw = FunctionsStringSearchToString, NameJSONSExtractRaw>; +struct NameSimpleJSONExtractRaw { static constexpr auto name = "SimpleJSONExtractRaw"; }; +using FunctionSimpleJSONExtractRaw = FunctionsStringSearchToString, NameSimpleJSONExtractRaw>; void registerFunctionVisitParamExtractRaw(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractString.cpp b/src/Functions/visitParamExtractString.cpp index a7df64e6be4..c1f2f3eb7b0 100644 --- a/src/Functions/visitParamExtractString.cpp +++ b/src/Functions/visitParamExtractString.cpp @@ -20,13 +20,13 @@ struct ExtractString struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; }; using FunctionVisitParamExtractString = FunctionsStringSearchToString, NameVisitParamExtractString>; -struct NameJSONSExtractString { static constexpr auto name = "JSONSExtractString"; }; -using FunctionJSONSExtractString = FunctionsStringSearchToString, NameJSONSExtractString>; +struct NameSimpleJSONExtractString { static constexpr auto name = "SimpleJSONExtractString"; }; +using FunctionSimpleJSONExtractString = FunctionsStringSearchToString, NameSimpleJSONExtractString>; void registerFunctionVisitParamExtractString(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamExtractUInt.cpp b/src/Functions/visitParamExtractUInt.cpp index bf7bf211b7d..3f6eb87c2d9 100644 --- a/src/Functions/visitParamExtractUInt.cpp +++ b/src/Functions/visitParamExtractUInt.cpp @@ -9,14 +9,14 @@ namespace DB struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; }; using FunctionVisitParamExtractUInt = FunctionsStringSearch>, NameVisitParamExtractUInt>; -struct NameJSONSExtractUInt { static constexpr auto name = "JSONSExtractUInt"; }; -using FunctionJSONSExtractUInt = FunctionsStringSearch>, NameJSONSExtractUInt>; +struct NameSimpleJSONExtractUInt { static constexpr auto name = "SimpleJSONExtractUInt"; }; +using FunctionSimpleJSONExtractUInt = FunctionsStringSearch>, NameSimpleJSONExtractUInt>; void registerFunctionVisitParamExtractUInt(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/visitParamHas.cpp b/src/Functions/visitParamHas.cpp index a6dc2df41ba..88a63e57aca 100644 --- a/src/Functions/visitParamHas.cpp +++ b/src/Functions/visitParamHas.cpp @@ -19,13 +19,13 @@ struct HasParam struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; }; using FunctionVisitParamHas = FunctionsStringSearch, NameVisitParamHas>; -struct NameJSONSHas { static constexpr auto name = "JSONSHas"; }; -using FunctionJSONSHas = FunctionsStringSearch, NameJSONSHas>; +struct NameSimpleJSONHas { static constexpr auto name = "SimpleJSONHas"; }; +using FunctionSimpleJSONHas = FunctionsStringSearch, NameSimpleJSONHas>; void registerFunctionVisitParamHas(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql index f6840947ae7..c337509fd36 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql @@ -16,10 +16,10 @@ SELECT visitParamExtractRaw('{"myparam": "["}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": ["]", "2", "3"], "other":123}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": {"nested" : [1,2,3]}, "other":123}', 'myparam'); -SELECT JSONExtractInt('{"myparam":-1}', 'myparam'); -SELECT JSONExtractUInt('{"myparam":-1}', 'myparam'); -SELECT JSONExtractFloat('{"myparam":null}', 'myparam'); -SELECT JSONExtractFloat('{"myparam":-1}', 'myparam'); -SELECT JSONExtractBool('{"myparam":true}', 'myparam'); -SELECT JSONExtractString('{"myparam":"test_string"}', 'myparam'); -SELECT JSONExtractString('{"myparam":"test\\"string"}', 'myparam'); +SELECT SimpleJSONExtractInt('{"myparam":-1}', 'myparam'); +SELECT SimpleJSONExtractUInt('{"myparam":-1}', 'myparam'); +SELECT SimpleJSONExtractFloat('{"myparam":null}', 'myparam'); +SELECT SimpleJSONExtractFloat('{"myparam":-1}', 'myparam'); +SELECT SimpleJSONExtractBool('{"myparam":true}', 'myparam'); +SELECT SimpleJSONExtractString('{"myparam":"test_string"}', 'myparam'); +SELECT SimpleJSONExtractString('{"myparam":"test\\"string"}', 'myparam'); From 9d9feb8c5da4a880244fad7d7f869c6a843d6a20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 10:16:15 +0300 Subject: [PATCH 185/266] Fix GitHub --- .../01781_token_extractor_buffer_overflow.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference index aeba736d914..aa47d0d46d4 100644 --- a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.reference @@ -1,2 +1,2 @@ -0 -0 +0 +0 From 2a44a808a6dc4a7e75bf478ef17f9683afbf1d6d Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 10:44:26 +0300 Subject: [PATCH 186/266] ppc64le: Fix compile and stack trace --- src/Common/StackTrace.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index c4cf7f11e68..9e81cdddbda 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -35,7 +35,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext else error << "Address: " << info.si_addr; -#if defined(__x86_64__) && !defined(__FreeBSD__) && !defined(__APPLE__) && !defined(__arm__) +#if defined(__x86_64__) && !defined(__FreeBSD__) && !defined(__APPLE__) && !defined(__arm__) && !defined(__powerpc__) auto err_mask = context.uc_mcontext.gregs[REG_ERR]; if ((err_mask & 0x02)) error << " Access: write."; @@ -186,6 +186,8 @@ static void * getCallerAddress(const ucontext_t & context) # endif #elif defined(__aarch64__) return reinterpret_cast(context.uc_mcontext.pc); +#elif defined(__powerpc64__) + return reinterpret_cast(context.uc_mcontext.gp_regs[PT_NIP]); #else return nullptr; #endif From 42ae39f78f9097b8c501bd16daaea79fb764c8ac Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 1 Apr 2021 12:13:36 +0300 Subject: [PATCH 187/266] Updated tests --- src/Dictionaries/DirectDictionary.cpp | 20 ++++---------------- tests/performance/flat_dictionary.xml | 3 ++- 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 47698949924..96ef259106a 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -45,8 +45,6 @@ ColumnPtr DirectDictionary::getColumn( DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); const auto requested_keys = extractor.extractAllKeys(); - Field value; - const DictionaryAttribute & attribute = dict_struct.getAttribute(attribute_name, result_type); DefaultValueProvider default_value_provider(attribute.null_value, default_values_column); @@ -54,6 +52,7 @@ ColumnPtr DirectDictionary::getColumn( key_to_fetched_index.reserve(requested_keys.size()); auto fetched_from_storage = attribute.type->createColumn(); + size_t fetched_key_index = 0; size_t requested_attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; @@ -67,14 +66,9 @@ ColumnPtr DirectDictionary::getColumn( while (const auto block = stream->read()) { - auto block_columns = block.getColumns(); - /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) - { - block_key_columns.emplace_back(*block_columns.begin()); - block_columns.erase(block_columns.begin()); - } + block_key_columns.emplace_back(block.safeGetByPosition(i).column); DictionaryKeysExtractor block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena()); auto block_keys = block_keys_extractor.extractAllKeys(); @@ -85,7 +79,6 @@ ColumnPtr DirectDictionary::getColumn( for (size_t block_key_index = 0; block_key_index < block_keys.size(); ++block_key_index) { auto block_key = block_keys[block_key_index]; - key_to_fetched_index[block_key] = fetched_key_index; ++fetched_key_index; } @@ -98,10 +91,10 @@ ColumnPtr DirectDictionary::getColumn( Field value_to_insert; size_t requested_keys_size = requested_keys.size(); + auto result = fetched_from_storage->cloneEmpty(); result->reserve(requested_keys_size); - for (size_t requested_key_index = 0; requested_key_index < requested_keys_size; ++requested_key_index) { const auto requested_key = requested_keys[requested_key_index]; @@ -153,14 +146,9 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys(const Columns & while (const auto block = stream->read()) { - auto block_columns = block.getColumns(); - /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) - { - block_key_columns.emplace_back(*block_columns.begin()); - block_columns.erase(block_columns.begin()); - } + block_key_columns.emplace_back(block.safeGetByPosition(i).column); DictionaryKeysExtractor block_keys_extractor(block_key_columns, arena_holder.getComplexKeyArena()); size_t block_keys_size = block_keys_extractor.getKeysSize(); diff --git a/tests/performance/flat_dictionary.xml b/tests/performance/flat_dictionary.xml index 6eb4bc2be8f..426aa929bbc 100644 --- a/tests/performance/flat_dictionary.xml +++ b/tests/performance/flat_dictionary.xml @@ -57,7 +57,8 @@ SELECT dictGet('default.simple_key_flat_dictionary', {column_name}, number) FROM system.numbers - LIMIT {elements_count}; + LIMIT {elements_count} + FORMAR Null; From 8f23d39f2604e788c417148136bbc24aeaa313be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Apr 2021 12:33:08 +0300 Subject: [PATCH 188/266] Update version_date.tsv after release 20.8.15.11 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 799492cdd90..3e1073b8529 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -39,6 +39,7 @@ v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 +v20.8.15.11-lts 2021-04-01 v20.8.14.4-lts 2021-03-03 v20.8.13.15-lts 2021-02-20 v20.8.12.2-lts 2021-01-16 From 2c522cf8bbf9215ae441ed39813210acd8210bee Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 1 Apr 2021 15:06:08 +0300 Subject: [PATCH 189/266] Update FunctionsExternalDictionaries.h --- src/Functions/FunctionsExternalDictionaries.h | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 121e693c759..5a1a124531a 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -773,7 +773,7 @@ private: 2. If column is already nullable we merge column null map with null map that we get from dict has. */ - auto dict_has_arguments = filterAttributesForDictHas(arguments); + auto dict_has_arguments = filterAttributeNameArgumentForDictHas(arguments); auto is_key_in_dictionary_column = dictionary_has_func_impl.executeImpl(dict_has_arguments, std::make_shared(), input_rows_count); auto is_key_in_dictionary_column_mutable = is_key_in_dictionary_column->assumeMutable(); ColumnVector & is_key_in_dictionary_column_typed = assert_cast &>(*is_key_in_dictionary_column_mutable); @@ -839,14 +839,15 @@ private: null_map[i] = null_map[i] || null_map_to_add[i]; } - static ColumnsWithTypeAndName filterAttributesForDictHas(const ColumnsWithTypeAndName & arguments) + static ColumnsWithTypeAndName filterAttributeNameArgumentForDictHas(const ColumnsWithTypeAndName & arguments) { ColumnsWithTypeAndName dict_has_arguments; dict_has_arguments.reserve(arguments.size() - 1); + size_t attribute_name_argument_index = 1; for (size_t i = 0; i < arguments.size(); ++i) { - if (i == 1) + if (i == attribute_name_argument_index) continue; dict_has_arguments.emplace_back(arguments[i]); From ba64bc8557d259bcd043f919a6f1efb80378341f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 15:21:29 +0300 Subject: [PATCH 190/266] quick fix for inaccessible DatabaseReplicated --- src/Storages/System/StorageSystemClusters.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 25b432252f9..01e30ed9726 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -37,8 +37,22 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context const auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & name_and_database : databases) { - if (const auto * replicated = typeid_cast(name_and_database.second.get())) - writeCluster(res_columns, {name_and_database.first, replicated->getCluster()}); + if (const auto * replicated == typeid_cast(name_and_database.second.get())) + { + // A quick fix for stateless tests with DatabaseReplicated. Its ZK + // node can be destroyed at any time. If another test lists + // system.clusters to get client command line suggestions, it will + // get an error when trying to get the info about DB from ZK. + // Just ignore these inaccessible databases. A good example of a + // failing test is `01526_client_start_and_exit`. + try { + writeCluster(res_columns, {name_and_database.first, replicated->getCluster()}); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } } } From c7bf01fa33fa1f8043f97b40fe55d5acb641ca14 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 16:08:18 +0300 Subject: [PATCH 191/266] typo --- src/Storages/System/StorageSystemClusters.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 01e30ed9726..e20ce233190 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -37,7 +37,7 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context const auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & name_and_database : databases) { - if (const auto * replicated == typeid_cast(name_and_database.second.get())) + if (const auto * replicated = typeid_cast(name_and_database.second.get())) { // A quick fix for stateless tests with DatabaseReplicated. Its ZK // node can be destroyed at any time. If another test lists From 1a7011ae1200ad241fe19ecb4a622fc472a3f0c6 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Thu, 1 Apr 2021 16:22:50 +0300 Subject: [PATCH 192/266] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 6ef52614c21..754b714ff1c 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -481,7 +481,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## max_concurrent_queries {#max-concurrent-queries} -Определяет максимальное количество одновременно обрабатываемых запросов, связанных к семейством таблиц `MergeTree`. Запросы также могут быть ограничены настройками: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). +Определяет максимальное количество одновременно обрабатываемых запросов, связанных с таблицей семейства `MergeTree`. Запросы также могут быть ограничены настройками: [max_concurrent_queries_for_all_users](#max-concurrent-queries-for-all-users), [min_marks_to_honor_max_concurrent_queries](#min-marks-to-honor-max-concurrent-queries). !!! info "Примечание" Параметры этих настроек могут быть изменены во время выполнения запросов и вступят в силу немедленно. Запросы, которые уже запущены, выполнятся без изменений. @@ -1182,4 +1182,3 @@ ClickHouse использует ZooKeeper для хранения метадан ``` - From fd2a053bddd7377a1414eb67d1fe02ce7cb413ac Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Thu, 1 Apr 2021 16:24:21 +0300 Subject: [PATCH 193/266] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 754b714ff1c..109146d27f4 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -519,7 +519,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## min_marks_to_honor_max_concurrent_queries {#min-marks-to-honor-max-concurrent-queries} -Определяет минимальное количество меток, которое считает запрос, для применения настройки [max_concurrent_queries](#max-concurrent-queries). +Определяет минимальное количество засечек, считываемых запросом для применения настройки [max_concurrent_queries](#max-concurrent-queries). Возможные значения: From 19550d51739741ff6cc36d3784d484caace3af66 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 1 Apr 2021 16:26:13 +0300 Subject: [PATCH 194/266] Initialize pointer with nullptr --- src/Server/HTTP/HTTPServerResponse.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTP/HTTPServerResponse.h b/src/Server/HTTP/HTTPServerResponse.h index 32c74d40b13..f5b7a70dc79 100644 --- a/src/Server/HTTP/HTTPServerResponse.h +++ b/src/Server/HTTP/HTTPServerResponse.h @@ -62,7 +62,7 @@ public: private: Poco::Net::HTTPServerSession & session; - HTTPServerRequest * request; + HTTPServerRequest * request = nullptr; std::shared_ptr stream; std::shared_ptr header_stream; }; From 39c3efdb0071fb1d48daf44fb546e229786f7fcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 16:38:04 +0300 Subject: [PATCH 195/266] Fix build --- src/Storages/tests/gtest_SplitTokenExtractor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_SplitTokenExtractor.cpp b/src/Storages/tests/gtest_SplitTokenExtractor.cpp index b5a26c9cd8e..ee6a55f50b8 100644 --- a/src/Storages/tests/gtest_SplitTokenExtractor.cpp +++ b/src/Storages/tests/gtest_SplitTokenExtractor.cpp @@ -61,12 +61,12 @@ TEST_P(SplitTokenExtractorTest, next) for (const auto & expected_token : param.tokens) { SCOPED_TRACE(++i); - ASSERT_TRUE(token_extractor.next(data->data(), data->size(), &pos, &token_start, &token_len)); + ASSERT_TRUE(token_extractor.nextInColumn(data->data(), data->size(), &pos, &token_start, &token_len)); EXPECT_EQ(expected_token, std::string_view(data->data() + token_start, token_len)) << " token_start:" << token_start << " token_len: " << token_len; } - ASSERT_FALSE(token_extractor.next(data->data(), data->size(), &pos, &token_start, &token_len)) + ASSERT_FALSE(token_extractor.nextInColumn(data->data(), data->size(), &pos, &token_start, &token_len)) << "\n\t=> \"" << param.source.substr(token_start, token_len) << "\"" << "\n\t" << token_start << ", " << token_len << ", " << pos << ", " << data->size(); } From 1b613d454fe8432bc26f0a1733bda78b36afc319 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 1 Apr 2021 16:40:52 +0300 Subject: [PATCH 196/266] Fix flapping test_s3_zero_copy_replication --- tests/integration/test_s3_zero_copy_replication/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index d4c3ae06b72..5bc30ab1d6b 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -96,7 +96,7 @@ def test_s3_zero_copy_on_hybrid_storage(cluster): node1.query( """ CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') + ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}') ORDER BY id SETTINGS storage_policy='hybrid' """ @@ -131,3 +131,6 @@ def test_s3_zero_copy_on_hybrid_storage(cluster): assert node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')" assert node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')" + + node1.query("DROP TABLE IF EXISTS hybrid_test NO DELAY") + node2.query("DROP TABLE IF EXISTS hybrid_test NO DELAY") From 6f65a72112791f4d75c8e577c447d1d34bca9e58 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 1 Apr 2021 17:30:33 +0300 Subject: [PATCH 197/266] Remove socket from epoll before cancelling packet receiver to prevent race --- src/Client/HedgedConnections.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index a163ceba4a2..8455ef3117e 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -521,14 +521,17 @@ void HedgedConnections::processNewReplicaState(HedgedConnectionsFactory::State s void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { + /// It's important to remove file descriptor from epoll exactly before cancelling packet_receiver, + /// because otherwise another thread can try to receive a packet, get this file descriptor + /// from epoll and resume cancelled packet_receiver. + epoll.remove(replica.packet_receiver->getFileDescriptor()); + epoll.remove(replica.change_replica_timeout.getDescriptor()); + replica.packet_receiver->cancel(); replica.change_replica_timeout.reset(); - epoll.remove(replica.packet_receiver->getFileDescriptor()); --offset_states[fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset].active_connection_count; fd_to_replica_location.erase(replica.packet_receiver->getFileDescriptor()); - - epoll.remove(replica.change_replica_timeout.getDescriptor()); timeout_fd_to_replica_location.erase(replica.change_replica_timeout.getDescriptor()); --active_connection_count; From 4006be502b644e306e8ede3855e965b5b2a2f02b Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 1 Apr 2021 18:09:10 +0300 Subject: [PATCH 198/266] Add test --- .../0_stateless/01783_http_chunk_size.reference | 1 + .../queries/0_stateless/01783_http_chunk_size.sh | 15 +++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01783_http_chunk_size.reference create mode 100755 tests/queries/0_stateless/01783_http_chunk_size.sh diff --git a/tests/queries/0_stateless/01783_http_chunk_size.reference b/tests/queries/0_stateless/01783_http_chunk_size.reference new file mode 100644 index 00000000000..e454a00607c --- /dev/null +++ b/tests/queries/0_stateless/01783_http_chunk_size.reference @@ -0,0 +1 @@ +1234567890 1234567890 1234567890 1234567890 diff --git a/tests/queries/0_stateless/01783_http_chunk_size.sh b/tests/queries/0_stateless/01783_http_chunk_size.sh new file mode 100755 index 00000000000..80b8bfa9e77 --- /dev/null +++ b/tests/queries/0_stateless/01783_http_chunk_size.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "DROP TABLE IF EXISTS table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- +echo "CREATE TABLE table (a String) ENGINE Memory()" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- + +# NOTE: suppose that curl sends everything in a single chunk - there are no options to force the chunk-size. +echo "SET max_query_size=44" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- +echo -ne "INSERT INTO TABLE table FORMAT TabSeparated 1234567890 1234567890 1234567890 1234567890\n" | ${CLICKHOUSE_CURL} -H "Transfer-Encoding: chunked" -sS "${CLICKHOUSE_URL}&session_id=01783" --data-binary @- + +echo "SELECT * from table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- +echo "DROP TABLE table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- From eb127ded950e6b0d5fe383ffa3cfc786f1be5446 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 18:12:51 +0300 Subject: [PATCH 199/266] ppc64le: Fix config and timezone embedding. --- contrib/cctz-cmake/CMakeLists.txt | 17 ++++++++++++----- programs/server/CMakeLists.txt | 15 ++++++++++----- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 90e33dc9f62..a3869478347 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -97,12 +97,19 @@ if (NOT EXTERNAL_CCTZ_LIBRARY_FOUND OR NOT EXTERNAL_CCTZ_LIBRARY_WORKS) set(TZ_OBJS ${TZ_OBJS} ${TZ_OBJ}) # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake - add_custom_command(OUTPUT ${TZ_OBJ} - COMMAND cp ${TZDIR}/${TIMEZONE} ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID} - COMMAND cd ${CMAKE_CURRENT_BINARY_DIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} + # PPC64LE fails to do this with objcopy, use ld or lld instead + if (ARCH_PPC64LE) + add_custom_command(OUTPUT ${TZ_OBJ} + COMMAND cp ${TZDIR}/${TIMEZONE} ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID} + COMMAND cd ${CMAKE_CURRENT_BINARY_DIR} && ${CMAKE_LINKER} -m elf64lppc -r -b binary -o ${TZ_OBJ} ${TIMEZONE_ID} + COMMAND rm ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID}) + else() + add_custom_command(OUTPUT ${TZ_OBJ} + COMMAND cp ${TZDIR}/${TIMEZONE} ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID} + COMMAND cd ${CMAKE_CURRENT_BINARY_DIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} --rename-section .data=.rodata,alloc,load,readonly,data,contents ${TIMEZONE_ID} ${TZ_OBJ} - COMMAND rm ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID}) - + COMMAND rm ${CMAKE_CURRENT_BINARY_DIR}/${TIMEZONE_ID}) + endif() set_source_files_properties(${TZ_OBJ} PROPERTIES EXTERNAL_OBJECT true GENERATED true) endforeach(TIMEZONE) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 198d9081168..697851b294b 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -42,11 +42,16 @@ if (OS_LINUX) set(RESOURCE_OBJS ${RESOURCE_OBJS} ${RESOURCE_OBJ}) # https://stackoverflow.com/questions/14776463/compile-and-add-an-object-file-from-a-binary-with-cmake - add_custom_command(OUTPUT ${RESOURCE_OBJ} - COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} ${RESOURCE_FILE} ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ} - COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents - ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ}) - + # PPC64LE fails to do this with objcopy, use ld or lld instead + if (ARCH_PPC64LE) + add_custom_command(OUTPUT ${RESOURCE_OBJ} + COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${CMAKE_LINKER} -m elf64lppc -r -b binary -o ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ} ${RESOURCE_FILE}) + else() + add_custom_command(OUTPUT ${RESOURCE_OBJ} + COMMAND cd ${CMAKE_CURRENT_SOURCE_DIR} && ${OBJCOPY_PATH} -I binary ${OBJCOPY_ARCH_OPTIONS} ${RESOURCE_FILE} ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ} + COMMAND ${OBJCOPY_PATH} --rename-section .data=.rodata,alloc,load,readonly,data,contents + ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ} ${CMAKE_CURRENT_BINARY_DIR}/${RESOURCE_OBJ}) + endif() set_source_files_properties(${RESOURCE_OBJ} PROPERTIES EXTERNAL_OBJECT true GENERATED true) endforeach(RESOURCE_FILE) From 2d83cb38a3d0ded4759f55931d19b0280ab625c4 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 18:25:42 +0300 Subject: [PATCH 200/266] ppc64le: Enable building with jemalloc --- contrib/jemalloc-cmake/CMakeLists.txt | 6 +- .../internal/jemalloc_internal_defs.h.in | 367 ++++++++++++++++++ 2 files changed, 371 insertions(+), 2 deletions(-) create mode 100644 contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index b8a6474413a..73afa99f1d8 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ -if (SANITIZE OR NOT (ARCH_AMD64 OR ARCH_ARM) OR NOT (OS_LINUX OR OS_FREEBSD OR OS_DARWIN)) +if (SANITIZE OR NOT (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE) OR NOT (OS_LINUX OR OS_FREEBSD OR OS_DARWIN)) if (ENABLE_JEMALLOC) message (${RECONFIGURE_MESSAGE_LEVEL} - "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64 or aarch64 on linux or freebsd.") + "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used with x86_64, aarch64 or ppc64le on linux or freebsd.") endif() set (ENABLE_JEMALLOC OFF) else() @@ -107,6 +107,8 @@ if (ARCH_AMD64) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") elseif (ARCH_ARM) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") +elseif (ARCH_PPC64LE) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_ppc64le") else () message (FATAL_ERROR "internal jemalloc: This arch is not supported") endif () diff --git a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..8068861041f --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,367 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#define JEMALLOC_OVERRIDE___LIBC_CALLOC +#define JEMALLOC_OVERRIDE___LIBC_FREE +#define JEMALLOC_OVERRIDE___LIBC_MALLOC +#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN +#define JEMALLOC_OVERRIDE___LIBC_REALLOC +#define JEMALLOC_OVERRIDE___LIBC_VALLOC +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS 1 + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS 1 +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS 1 + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS 1 +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS 1 + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +// #define JEMALLOC_HAVE_SECURE_GETENV + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE 1 + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC 1 + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 16 + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H 1 + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT 1 + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +#define JEMALLOC_GLIBC_MALLOC_HOOK + +/* glibc memalign hook. */ +#define JEMALLOC_GLIBC_MEMALIGN_HOOK + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD 1 + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC 1 + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From bfd042a97977c28a8f11203fe72e1264fc9cf70f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Apr 2021 18:26:33 +0300 Subject: [PATCH 201/266] Lower scale of a test --- .../00159_parallel_formatting_http.reference | 12 ++++++------ .../1_stateful/00159_parallel_formatting_http.sh | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/1_stateful/00159_parallel_formatting_http.reference b/tests/queries/1_stateful/00159_parallel_formatting_http.reference index 499a0b8a7c7..8eabf5d4f03 100644 --- a/tests/queries/1_stateful/00159_parallel_formatting_http.reference +++ b/tests/queries/1_stateful/00159_parallel_formatting_http.reference @@ -1,12 +1,12 @@ TSV, false -8a984bbbfb127c430f67173f5371c6cb - +6e4ce4996dd0e036d27cb0d2166c8e59 - TSV, true -8a984bbbfb127c430f67173f5371c6cb - +6e4ce4996dd0e036d27cb0d2166c8e59 - CSV, false -ea1c740f03f5dcc43a3044528ad0a98f - +ab6b3616f31e8a952c802ca92562e418 - CSV, true -ea1c740f03f5dcc43a3044528ad0a98f - +ab6b3616f31e8a952c802ca92562e418 - JSONCompactEachRow, false -ba1081a754a06ef6563840b2d8d4d327 - +1651b540b43bd6c62446f4c340bf13c7 - JSONCompactEachRow, true -ba1081a754a06ef6563840b2d8d4d327 - +1651b540b43bd6c62446f4c340bf13c7 - diff --git a/tests/queries/1_stateful/00159_parallel_formatting_http.sh b/tests/queries/1_stateful/00159_parallel_formatting_http.sh index 8fd8c15b7c7..a4e68de6a3f 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_http.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_http.sh @@ -10,8 +10,8 @@ FORMATS=('TSV' 'CSV' 'JSONCompactEachRow') for format in "${FORMATS[@]}" do echo "$format, false"; - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+Format+$format&output_format_parallel_formatting=false" -d' ' | md5sum + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=false" -d' ' | md5sum echo "$format, true"; - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+Format+$format&output_format_parallel_formatting=true" -d' ' | md5sum + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=SELECT+ClientEventTime+as+a,MobilePhoneModel+as+b,ClientIP6+as+c+FROM+test.hits+ORDER+BY+a,b,c+LIMIT+1000000+Format+$format&output_format_parallel_formatting=true" -d' ' | md5sum done From 00ed52e947e7e2b90e434171527242510e1ad5a5 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 18:38:16 +0300 Subject: [PATCH 202/266] Fix finding libclang_rt.builtins on Fedora\RHEL\CentOS on ppc64le --- cmake/linux/default_libs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index d3a727e9cb8..c1e4d450389 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -6,7 +6,7 @@ set (DEFAULT_LIBS "-nodefaultlibs") # We need builtins from Clang's RT even without libcxx - for ubsan+int128. # See https://bugs.llvm.org/show_bug.cgi?id=16404 if (COMPILER_CLANG AND NOT (CMAKE_CROSSCOMPILING AND ARCH_AARCH64)) - execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) else () set (BUILTINS_LIBRARY "-lgcc") endif () From d006c5532e464fa35903bab9b951b5de39377b67 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 18:56:32 +0300 Subject: [PATCH 203/266] fix window frame offset check and add more tests --- src/Interpreters/convertFieldToType.cpp | 7 +- src/Processors/Transforms/WindowTransform.cpp | 34 +++++-- .../01591_window_functions.reference | 89 +++++++++++++------ .../0_stateless/01591_window_functions.sql | 35 +++++++- 4 files changed, 129 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 90b840ce8bd..ed920539bea 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -397,8 +397,11 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); Field converted = convertFieldToType(from_value, to_type, from_type_hint); if (!is_null && converted.isNull()) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Cannot convert value{}: it cannot be represented as {}", - from_type_hint ? " from " + from_type_hint->getName() : "", to_type.getName()); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Cannot convert value '{}'{}: it cannot be represented as {}", + toString(from_value), + from_type_hint ? " from " + from_type_hint->getName() : "", + to_type.getName()); return converted; } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 4a5282c1e6b..16d028f0fc1 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -48,7 +49,10 @@ static int compareValuesWithOffset(const IColumn * _compared_column, _compared_column); const auto * reference_column = assert_cast( _reference_column); - const auto offset = _offset.get(); + // Note that the storage type of offset returned by get<> is different, so + // we need to specify the type explicitly. + const typename ColumnType::ValueType offset + = _offset.get(); assert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); @@ -62,32 +66,32 @@ static int compareValuesWithOffset(const IColumn * _compared_column, reference_value_data.data); bool is_overflow; - bool overflow_to_negative; if (offset_is_preceding) { is_overflow = __builtin_sub_overflow(reference_value, offset, &reference_value); - overflow_to_negative = offset > 0; } else { is_overflow = __builtin_add_overflow(reference_value, offset, &reference_value); - overflow_to_negative = offset < 0; } // fmt::print(stderr, -// "compared [{}] = {}, ref [{}] = {}, offset {} preceding {} overflow {} to negative {}\n", +// "compared [{}] = {}, old ref {}, shifted ref [{}] = {}, offset {} preceding {} overflow {} to negative {}\n", // compared_row, toString(compared_value), +// // fmt doesn't like char8_t. +// static_cast(unalignedLoad(reference_value_data.data)), // reference_row, toString(reference_value), // toString(offset), offset_is_preceding, -// is_overflow, overflow_to_negative); +// is_overflow, offset_is_preceding); if (is_overflow) { - if (overflow_to_negative) + if (offset_is_preceding) { // Overflow to the negative, [compared] must be greater. + // We know that because offset is >= 0. return 1; } else @@ -263,6 +267,14 @@ WindowTransform::WindowTransform(const Block & input_header_, window_description.frame.begin_offset = convertFieldToTypeOrThrow( window_description.frame.begin_offset, *entry.type); + + if (applyVisitor(FieldVisitorAccurateLess{}, + window_description.frame.begin_offset, Field(0))) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window frame start offset must be nonnegative, {} given", + window_description.frame.begin_offset); + } } if (window_description.frame.end_type == WindowFrame::BoundaryType::Offset) @@ -270,6 +282,14 @@ WindowTransform::WindowTransform(const Block & input_header_, window_description.frame.end_offset = convertFieldToTypeOrThrow( window_description.frame.end_offset, *entry.type); + + if (applyVisitor(FieldVisitorAccurateLess{}, + window_description.frame.end_offset, Field(0))) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Window frame start offset must be nonnegative, {} given", + window_description.frame.end_offset); + } } } } diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 14e5889a811..55b945d3816 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -771,6 +771,28 @@ order by x; 125 124 127 4 126 125 127 3 127 126 127 2 +-- We need large offsets to trigger overflow to positive direction, or +-- else the frame end runs into partition end w/o overflow and doesn't move +-- after that. The frame from this query is equivalent to the entire partition. +select x, min(x) over w, max(x) over w, count(x) over w +from ( + select toUInt8(if(mod(number, 2), + toInt64(255 - intDiv(number, 2)), + toInt64(intDiv(number, 2)))) x + from numbers(10) +) +window w as (order by x range between 255 preceding and 255 following) +order by x; +0 0 255 10 +1 0 255 10 +2 0 255 10 +3 0 255 10 +4 0 255 10 +251 0 255 10 +252 0 255 10 +253 0 255 10 +254 0 255 10 +255 0 255 10 -- RANGE OFFSET ORDER BY DESC select x, min(x) over w, max(x) over w, count(x) over w from ( select toUInt8(number) x from numbers(11)) t @@ -1004,6 +1026,8 @@ from numbers(5); 3 \N -- variants of lag/lead that respect the frame select number, p, pp, + lagInFrame(number) over w as lag1, + lagInFrame(number, number - pp) over w as lag2, lagInFrame(number, number - pp, number * 11) over w as lag, leadInFrame(number, number - pp, number * 11) over w as lead from (select number, intDiv(number, 5) p, p * 5 pp from numbers(16)) @@ -1012,22 +1036,22 @@ window w as (partition by p order by number order by number settings max_block_size = 3; ; -0 0 0 0 0 -1 0 0 0 2 -2 0 0 0 4 -3 0 0 0 33 -4 0 0 0 44 -5 1 5 5 5 -6 1 5 5 7 -7 1 5 5 9 -8 1 5 5 88 -9 1 5 5 99 -10 2 10 10 10 -11 2 10 10 12 -12 2 10 10 14 -13 2 10 10 143 -14 2 10 10 154 -15 3 15 15 15 +0 0 0 0 0 0 0 +1 0 0 0 0 0 2 +2 0 0 1 0 0 4 +3 0 0 2 0 0 33 +4 0 0 3 0 0 44 +5 1 5 0 5 5 5 +6 1 5 5 5 5 7 +7 1 5 6 5 5 9 +8 1 5 7 5 5 88 +9 1 5 8 5 5 99 +10 2 10 0 10 10 10 +11 2 10 10 10 10 12 +12 2 10 11 10 10 14 +13 2 10 12 10 10 143 +14 2 10 13 10 10 154 +15 3 15 0 15 15 15 -- case-insensitive SQL-standard synonyms for any and anyLast select number, @@ -1054,14 +1078,29 @@ select count() over () from numbers(4) where number < 2; 2 -- floating point RANGE frame select - count(*) over (order by (toFloat32(number) as f32) range 5. preceding), - count(*) over (order by (toFloat64(number) as f64) range 5. preceding) + count(*) over (order by toFloat32(number) range 5. preceding), + count(*) over (order by toFloat64(number) range 5. preceding), + count(*) over (order by toFloat32(number) range between current row and 5. following), + count(*) over (order by toFloat64(number) range between current row and 5. following) from numbers(7) ; -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -6 6 +1 1 6 6 +2 2 6 6 +3 3 5 5 +4 4 4 4 +5 5 3 3 +6 6 2 2 +6 6 1 1 +-- negative offsets should not be allowed +select count() over (order by toInt64(number) range between -1 preceding and unbounded following) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between -1 following and unbounded following) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between unbounded preceding and -1 preceding) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between unbounded preceding and -1 following) from numbers(1); -- { serverError 36 } +---- a test with aggregate function that allocates memory in arena +select sum(a[length(a)]) +from ( + select groupArray(number) over (partition by modulo(number, 11) + order by modulo(number, 1111)) a + from numbers_mt(10000) +) settings max_block_size = 7; +50091657 diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 30847e09246..412cc1fe79f 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -242,6 +242,19 @@ from ( window w as (order by x range between 1 preceding and 2 following) order by x; +-- We need large offsets to trigger overflow to positive direction, or +-- else the frame end runs into partition end w/o overflow and doesn't move +-- after that. The frame from this query is equivalent to the entire partition. +select x, min(x) over w, max(x) over w, count(x) over w +from ( + select toUInt8(if(mod(number, 2), + toInt64(255 - intDiv(number, 2)), + toInt64(intDiv(number, 2)))) x + from numbers(10) +) +window w as (order by x range between 255 preceding and 255 following) +order by x; + -- RANGE OFFSET ORDER BY DESC select x, min(x) over w, max(x) over w, count(x) over w from ( select toUInt8(number) x from numbers(11)) t @@ -349,6 +362,8 @@ from numbers(5); -- variants of lag/lead that respect the frame select number, p, pp, + lagInFrame(number) over w as lag1, + lagInFrame(number, number - pp) over w as lag2, lagInFrame(number, number - pp, number * 11) over w as lag, leadInFrame(number, number - pp, number * 11) over w as lead from (select number, intDiv(number, 5) p, p * 5 pp from numbers(16)) @@ -374,7 +389,23 @@ select count() over () from numbers(4) where number < 2; -- floating point RANGE frame select - count(*) over (order by (toFloat32(number) as f32) range 5. preceding), - count(*) over (order by (toFloat64(number) as f64) range 5. preceding) + count(*) over (order by toFloat32(number) range 5. preceding), + count(*) over (order by toFloat64(number) range 5. preceding), + count(*) over (order by toFloat32(number) range between current row and 5. following), + count(*) over (order by toFloat64(number) range between current row and 5. following) from numbers(7) ; + +-- negative offsets should not be allowed +select count() over (order by toInt64(number) range between -1 preceding and unbounded following) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between -1 following and unbounded following) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between unbounded preceding and -1 preceding) from numbers(1); -- { serverError 36 } +select count() over (order by toInt64(number) range between unbounded preceding and -1 following) from numbers(1); -- { serverError 36 } + +---- a test with aggregate function that allocates memory in arena +select sum(a[length(a)]) +from ( + select groupArray(number) over (partition by modulo(number, 11) + order by modulo(number, 1111)) a + from numbers_mt(10000) +) settings max_block_size = 7; From 9b38ed1f92a9d6d911667a5087dcd88a013d5376 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 19:55:35 +0300 Subject: [PATCH 204/266] ppc64le: Fix CMake error about internal CMake variable CMAKE_ASM_COMPILE_OBJECT not set --- CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 81beb4d1947..5d716985c46 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -39,6 +39,8 @@ else() set(RECONFIGURE_MESSAGE_LEVEL STATUS) endif() +enable_language(C CXX ASM) + include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) From 0bb3b90c50e245f58663e107a4b006d857b33456 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 1 Apr 2021 20:06:49 +0300 Subject: [PATCH 205/266] Update database_replicated.xml --- tests/config/config.d/database_replicated.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index ed5845bad48..c2e62f9645a 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -21,6 +21,9 @@ 5000 10000 + 1000 + 2000 + 4000 trace false From 60e59464491409b959a63c87478bce924b3ba452 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 31 Mar 2021 01:49:15 +0300 Subject: [PATCH 206/266] Removed needless logging, fixed data part initialization while checking in the detached folder --- src/Storages/StorageReplicatedMergeTree.cpp | 40 +++++++++++++++------ 1 file changed, 29 insertions(+), 11 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 23a6abe2da7..b841debbad2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1359,8 +1359,6 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const MergeTreePartInfo actual_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); const String part_new_name = actual_part_info.getPartName(); - LOG_TRACE(log, "Trying to attach part {}, checksum {}", part_new_name, entry.part_checksum); - for (const DiskPtr & disk : getStoragePolicy()->getDisks()) for (const auto it = disk->iterateDirectory(relative_data_path + "detached/"); it->isValid(); it->next()) { @@ -1374,7 +1372,9 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const String part_path = "detached/" + part_old_name; const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); - MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, part_info, volume, part_path); + + /// actual_part_info is more recent than part_info so we use it + MergeTreeData::MutableDataPartPtr part = createPart(part_new_name, actual_part_info, volume, part_path); try { @@ -1442,7 +1442,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) Transaction transaction(*this); - renameTempPartAndAdd(part, nullptr, &transaction); + renameTempPartAndReplace(part, nullptr, &transaction); checkPartChecksumsAndCommit(transaction, part); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, @@ -1498,7 +1498,8 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) { - LOG_TRACE(log, "Executing log entry to merge parts {} to {}", boost::algorithm::join(entry.source_parts, ", "), entry.new_part_name); + LOG_TRACE(log, "Executing log entry to merge parts {} to {}", + fmt::join(entry.source_parts, ", "), entry.new_part_name); const auto storage_settings_ptr = getSettings(); @@ -1523,6 +1524,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) /// instead of doing exactly the same merge cluster-wise std::optional replica_to_execute_merge; bool replica_to_execute_merge_picked = false; + if (merge_strategy_picker.shouldMergeOnSingleReplica(entry)) { replica_to_execute_merge = merge_strategy_picker.pickReplicaToExecuteMerge(entry); @@ -1530,17 +1532,21 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) if (replica_to_execute_merge) { - LOG_DEBUG(log, "Prefer fetching part {} from replica {} due execute_merges_on_single_replica_time_threshold", + LOG_DEBUG(log, + "Prefer fetching part {} from replica {} due to execute_merges_on_single_replica_time_threshold", entry.new_part_name, replica_to_execute_merge.value()); + return false; } } DataPartsVector parts; bool have_all_parts = true; + for (const String & name : entry.source_parts) { DataPartPtr part = getActiveContainingPart(name); + if (!part) { have_all_parts = false; @@ -1623,8 +1629,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) if (storage_settings_ptr->allow_s3_zero_copy_replication) { - auto disk = reserved_space->getDisk(); - if (disk->getType() == DB::DiskType::Type::S3) + if (auto disk = reserved_space->getDisk(); disk->getType() == DB::DiskType::Type::S3) { if (merge_strategy_picker.shouldMergeOnSingleReplicaS3Shared(entry)) { @@ -1633,7 +1638,9 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) if (replica_to_execute_merge) { - LOG_DEBUG(log, "Prefer fetching part {} from replica {} due s3_execute_merges_on_single_replica_time_threshold", entry.new_part_name, replica_to_execute_merge.value()); + LOG_DEBUG(log, + "Prefer fetching part {} from replica {} due s3_execute_merges_on_single_replica_time_threshold", + entry.new_part_name, replica_to_execute_merge.value()); return false; } } @@ -1645,8 +1652,10 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) global_context.getMergeList().bookMergeWithTTL(); auto table_id = getStorageID(); + /// Add merge to list - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_merged_part); + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert( + table_id.database_name, table_id.table_name, future_merged_part); Transaction transaction(*this); MutableDataPartPtr part; @@ -1680,7 +1689,16 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) ProfileEvents::increment(ProfileEvents::DataAfterMergeDiffersFromReplica); - LOG_ERROR(log, "{}. Data after merge is not byte-identical to data on another replicas. There could be several reasons: 1. Using newer version of compression library after server update. 2. Using another compression method. 3. Non-deterministic compression algorithm (highly unlikely). 4. Non-deterministic merge algorithm due to logical error in code. 5. Data corruption in memory due to bug in code. 6. Data corruption in memory due to hardware issue. 7. Manual modification of source data after server startup. 8. Manual modification of checksums stored in ZooKeeper. 9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas. We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false)); + LOG_ERROR(log, + "{}. Data after merge is not byte-identical to data on another replicas. There could be several" + " reasons: 1. Using newer version of compression library after server update. 2. Using another" + " compression method. 3. Non-deterministic compression algorithm (highly unlikely). 4." + " Non-deterministic merge algorithm due to logical error in code. 5. Data corruption in memory due" + " to bug in code. 6. Data corruption in memory due to hardware issue. 7. Manual modification of" + " source data after server startup. 8. Manual modification of checksums stored in ZooKeeper. 9." + " Part format related settings like 'enable_mixed_granularity_parts' are different on different" + " replicas. We will download merged part from replica to force byte-identical result.", + getCurrentExceptionMessage(false)); write_part_log(ExecutionStatus::fromCurrentException()); From 3f57ed41ed0513e51fa2cbc879b09451e17547c5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 1 Apr 2021 20:16:27 +0300 Subject: [PATCH 207/266] Update 01591_window_functions.sql --- tests/queries/0_stateless/01591_window_functions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 412cc1fe79f..85856dd797d 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -406,6 +406,6 @@ select count() over (order by toInt64(number) range between unbounded preceding select sum(a[length(a)]) from ( select groupArray(number) over (partition by modulo(number, 11) - order by modulo(number, 1111)) a + order by modulo(number, 1111), number) a from numbers_mt(10000) ) settings max_block_size = 7; From ac97f7df29e2ddd4c79c78852a5bd160e3df734b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 1 Apr 2021 20:16:59 +0300 Subject: [PATCH 208/266] Update 01591_window_functions.reference --- tests/queries/0_stateless/01591_window_functions.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 55b945d3816..9067ee8d955 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -1100,7 +1100,7 @@ select count() over (order by toInt64(number) range between unbounded preceding select sum(a[length(a)]) from ( select groupArray(number) over (partition by modulo(number, 11) - order by modulo(number, 1111)) a + order by modulo(number, 1111), number) a from numbers_mt(10000) ) settings max_block_size = 7; -50091657 +49995000 From 31c8edf06a0226035b3e1268aef01b2803453eae Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 20:57:31 +0300 Subject: [PATCH 209/266] disable parallel run for DatabaseReplicated --- docker/test/stateless/run.sh | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 046cfb62608..20132eafb75 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -74,13 +74,17 @@ function run_tests() ADDITIONAL_OPTIONS+=('--order=random') ADDITIONAL_OPTIONS+=('--skip') ADDITIONAL_OPTIONS+=('00000_no_tests_to_skip') + # Note that flaky check must be ran in parallel, but for now we run + # everything in parallel except DatabaseReplicated. See below. fi - ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('8') - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') + else + # Too many tests fail for DatabaseReplicated in parallel. All other + # configurations are OK. + ADDITIONAL_OPTIONS+=('--jobs') + ADDITIONAL_OPTIONS+=('8') fi clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ From bd7cc80d4b0ac1a6afde214c73ad522c9cec7394 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 21:02:52 +0300 Subject: [PATCH 210/266] try clang 11 in fast test --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 64be52d8e30..1f96743e7d1 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -1,7 +1,7 @@ # docker build -t yandex/clickhouse-fasttest . FROM ubuntu:20.04 -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ From 8e26998376ab765f2de89836a1b3bf55e9b519ca Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 21:05:09 +0300 Subject: [PATCH 211/266] fixup --- docker/test/fasttest/run.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c8bfce3848d..1ebd29ebdf7 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -8,6 +8,9 @@ trap 'kill $(jobs -pr) ||:' EXIT # that we can run the "everything else" stage from the cloned source. stage=${stage:-} +# Compiler version, normally set by Dockerfile +export LLVM_VERSION=${LLVM_VERSION:-11} + # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about # it being undefined. Also read it as array so that we can pass an empty list @@ -215,7 +218,7 @@ function run_cmake ( cd "$FASTTEST_BUILD" - cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" + cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-${LLVM_VERSION} -DCMAKE_C_COMPILER=clang-${LLVM_VERSION} "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" ) } From fda70f97c36db792a169b5172090a4694dc7a201 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 21:44:43 +0300 Subject: [PATCH 212/266] not so deep --- docker/test/fasttest/run.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 1ebd29ebdf7..edda27344ae 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,21 +127,22 @@ continue function clone_root { - git clone https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" + git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( cd "$FASTTEST_SOURCE" if [ "$PULL_REQUEST_NUMBER" != "0" ]; then - if git fetch origin "+refs/pull/$PULL_REQUEST_NUMBER/merge"; then + if git fetch --depth 1 origin "+refs/pull/$PULL_REQUEST_NUMBER/merge"; then git checkout FETCH_HEAD echo 'Clonned merge head' else - git fetch origin "+refs/pull/$PULL_REQUEST_NUMBER/head" + git fetch --depth 1 origin "+refs/pull/$PULL_REQUEST_NUMBER/head" git checkout "$COMMIT_SHA" echo 'Checked out to commit' fi else if [ -v COMMIT_SHA ]; then + git fetch --depth 1 origin "$COMMIT_SHA" git checkout "$COMMIT_SHA" fi fi @@ -184,7 +185,7 @@ function clone_submodules ) git submodule sync - git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" + git submodule update --depth 1 --init --recursive "${SUBMODULES_TO_UPDATE[@]}" git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd @@ -218,7 +219,7 @@ function run_cmake ( cd "$FASTTEST_BUILD" - cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-${LLVM_VERSION} -DCMAKE_C_COMPILER=clang-${LLVM_VERSION} "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" + cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" ) } From b66e0401f689b51818bcf31a847ab2ceb4a43ddd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Apr 2021 23:07:01 +0300 Subject: [PATCH 213/266] Lowercase --- src/Functions/visitParamExtractBool.cpp | 2 +- src/Functions/visitParamExtractFloat.cpp | 2 +- src/Functions/visitParamExtractInt.cpp | 2 +- src/Functions/visitParamExtractRaw.cpp | 2 +- src/Functions/visitParamExtractString.cpp | 2 +- src/Functions/visitParamExtractUInt.cpp | 2 +- src/Functions/visitParamHas.cpp | 2 +- .../00539_functions_for_working_with_json.sql | 14 +++++++------- 8 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Functions/visitParamExtractBool.cpp b/src/Functions/visitParamExtractBool.cpp index a1b6c83180b..059115b5b13 100644 --- a/src/Functions/visitParamExtractBool.cpp +++ b/src/Functions/visitParamExtractBool.cpp @@ -19,7 +19,7 @@ struct ExtractBool struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; }; using FunctionVisitParamExtractBool = FunctionsStringSearch, NameVisitParamExtractBool>; -struct NameSimpleJSONExtractBool { static constexpr auto name = "SimpleJSONExtractBool"; }; +struct NameSimpleJSONExtractBool { static constexpr auto name = "simpleJSONExtractBool"; }; using FunctionSimpleJSONExtractBool = FunctionsStringSearch, NameSimpleJSONExtractBool>; void registerFunctionVisitParamExtractBool(FunctionFactory & factory) diff --git a/src/Functions/visitParamExtractFloat.cpp b/src/Functions/visitParamExtractFloat.cpp index 0ddfca95f1d..7a55cff365c 100644 --- a/src/Functions/visitParamExtractFloat.cpp +++ b/src/Functions/visitParamExtractFloat.cpp @@ -9,7 +9,7 @@ namespace DB struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; }; using FunctionVisitParamExtractFloat = FunctionsStringSearch>, NameVisitParamExtractFloat>; -struct NameSimpleJSONExtractFloat { static constexpr auto name = "SimpleJSONExtractFloat"; }; +struct NameSimpleJSONExtractFloat { static constexpr auto name = "simpleJSONExtractFloat"; }; using FunctionSimpleJSONExtractFloat = FunctionsStringSearch>, NameSimpleJSONExtractFloat>; void registerFunctionVisitParamExtractFloat(FunctionFactory & factory) diff --git a/src/Functions/visitParamExtractInt.cpp b/src/Functions/visitParamExtractInt.cpp index 094e3d966af..7c2188c10fc 100644 --- a/src/Functions/visitParamExtractInt.cpp +++ b/src/Functions/visitParamExtractInt.cpp @@ -9,7 +9,7 @@ namespace DB struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; }; using FunctionVisitParamExtractInt = FunctionsStringSearch>, NameVisitParamExtractInt>; -struct NameSimpleJSONExtractInt { static constexpr auto name = "SimpleJSONExtractInt"; }; +struct NameSimpleJSONExtractInt { static constexpr auto name = "simpleJSONExtractInt"; }; using FunctionSimpleJSONExtractInt = FunctionsStringSearch>, NameSimpleJSONExtractInt>; void registerFunctionVisitParamExtractInt(FunctionFactory & factory) diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index f1f73be2847..734fe107557 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -59,7 +59,7 @@ struct ExtractRaw struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; }; using FunctionVisitParamExtractRaw = FunctionsStringSearchToString, NameVisitParamExtractRaw>; -struct NameSimpleJSONExtractRaw { static constexpr auto name = "SimpleJSONExtractRaw"; }; +struct NameSimpleJSONExtractRaw { static constexpr auto name = "simpleJSONExtractRaw"; }; using FunctionSimpleJSONExtractRaw = FunctionsStringSearchToString, NameSimpleJSONExtractRaw>; void registerFunctionVisitParamExtractRaw(FunctionFactory & factory) diff --git a/src/Functions/visitParamExtractString.cpp b/src/Functions/visitParamExtractString.cpp index c1f2f3eb7b0..23f24b9e3b8 100644 --- a/src/Functions/visitParamExtractString.cpp +++ b/src/Functions/visitParamExtractString.cpp @@ -20,7 +20,7 @@ struct ExtractString struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; }; using FunctionVisitParamExtractString = FunctionsStringSearchToString, NameVisitParamExtractString>; -struct NameSimpleJSONExtractString { static constexpr auto name = "SimpleJSONExtractString"; }; +struct NameSimpleJSONExtractString { static constexpr auto name = "simpleJSONExtractString"; }; using FunctionSimpleJSONExtractString = FunctionsStringSearchToString, NameSimpleJSONExtractString>; void registerFunctionVisitParamExtractString(FunctionFactory & factory) diff --git a/src/Functions/visitParamExtractUInt.cpp b/src/Functions/visitParamExtractUInt.cpp index 3f6eb87c2d9..f5466a63b0d 100644 --- a/src/Functions/visitParamExtractUInt.cpp +++ b/src/Functions/visitParamExtractUInt.cpp @@ -9,7 +9,7 @@ namespace DB struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; }; using FunctionVisitParamExtractUInt = FunctionsStringSearch>, NameVisitParamExtractUInt>; -struct NameSimpleJSONExtractUInt { static constexpr auto name = "SimpleJSONExtractUInt"; }; +struct NameSimpleJSONExtractUInt { static constexpr auto name = "simpleJSONExtractUInt"; }; using FunctionSimpleJSONExtractUInt = FunctionsStringSearch>, NameSimpleJSONExtractUInt>; diff --git a/src/Functions/visitParamHas.cpp b/src/Functions/visitParamHas.cpp index 88a63e57aca..f4f377f9e8f 100644 --- a/src/Functions/visitParamHas.cpp +++ b/src/Functions/visitParamHas.cpp @@ -19,7 +19,7 @@ struct HasParam struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; }; using FunctionVisitParamHas = FunctionsStringSearch, NameVisitParamHas>; -struct NameSimpleJSONHas { static constexpr auto name = "SimpleJSONHas"; }; +struct NameSimpleJSONHas { static constexpr auto name = "simpleJSONHas"; }; using FunctionSimpleJSONHas = FunctionsStringSearch, NameSimpleJSONHas>; void registerFunctionVisitParamHas(FunctionFactory & factory) diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql index c337509fd36..31853e92262 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql @@ -16,10 +16,10 @@ SELECT visitParamExtractRaw('{"myparam": "["}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": ["]", "2", "3"], "other":123}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": {"nested" : [1,2,3]}, "other":123}', 'myparam'); -SELECT SimpleJSONExtractInt('{"myparam":-1}', 'myparam'); -SELECT SimpleJSONExtractUInt('{"myparam":-1}', 'myparam'); -SELECT SimpleJSONExtractFloat('{"myparam":null}', 'myparam'); -SELECT SimpleJSONExtractFloat('{"myparam":-1}', 'myparam'); -SELECT SimpleJSONExtractBool('{"myparam":true}', 'myparam'); -SELECT SimpleJSONExtractString('{"myparam":"test_string"}', 'myparam'); -SELECT SimpleJSONExtractString('{"myparam":"test\\"string"}', 'myparam'); +SELECT simpleJSONExtractInt('{"myparam":-1}', 'myparam'); +SELECT simpleJSONExtractUInt('{"myparam":-1}', 'myparam'); +SELECT simpleJSONExtractFloat('{"myparam":null}', 'myparam'); +SELECT simpleJSONExtractFloat('{"myparam":-1}', 'myparam'); +SELECT simpleJSONExtractBool('{"myparam":true}', 'myparam'); +SELECT simpleJSONExtractString('{"myparam":"test_string"}', 'myparam'); +SELECT simpleJSONExtractString('{"myparam":"test\\"string"}', 'myparam'); From 95cd9b0b37f709f4742ae2bb40151d45232ae0a3 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Apr 2021 23:28:53 +0300 Subject: [PATCH 214/266] cleanup --- docker/test/fasttest/Dockerfile | 6 +++--- docker/test/fasttest/run.sh | 7 +++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 1f96743e7d1..2864f7fc4da 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -43,20 +43,20 @@ RUN apt-get update \ clang-tidy-${LLVM_VERSION} \ cmake \ curl \ - lsof \ expect \ fakeroot \ - git \ gdb \ + git \ gperf \ lld-${LLVM_VERSION} \ llvm-${LLVM_VERSION} \ + lsof \ moreutils \ ninja-build \ psmisc \ python3 \ - python3-pip \ python3-lxml \ + python3-pip \ python3-requests \ python3-termcolor \ rename \ diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index edda27344ae..c711a253e23 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -134,16 +134,19 @@ function clone_root if [ "$PULL_REQUEST_NUMBER" != "0" ]; then if git fetch --depth 1 origin "+refs/pull/$PULL_REQUEST_NUMBER/merge"; then git checkout FETCH_HEAD - echo 'Clonned merge head' + echo "Checked out pull/$PULL_REQUEST_NUMBER/merge ($(git rev-parse FETCH_HEAD))" else git fetch --depth 1 origin "+refs/pull/$PULL_REQUEST_NUMBER/head" git checkout "$COMMIT_SHA" - echo 'Checked out to commit' + echo "Checked out nominal SHA $COMMIT_SHA for PR $PULL_REQUEST_NUMBER" fi else if [ -v COMMIT_SHA ]; then git fetch --depth 1 origin "$COMMIT_SHA" git checkout "$COMMIT_SHA" + echo "Checked out nominal SHA $COMMIT_SHA for master" + else + echo "Using default repository head $(git rev-parse HEAD)" fi fi ) From deb10ab1221bb19354c678ed670fdf80403484aa Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Thu, 1 Apr 2021 23:38:57 +0300 Subject: [PATCH 215/266] ppc64le: Fix compiling boost --- contrib/boost-cmake/CMakeLists.txt | 35 +++++++++++++++--------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index b9298f59f2b..0759935a7db 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -160,6 +160,12 @@ if (NOT EXTERNAL_BOOST_FOUND) enable_language(ASM) SET(ASM_OPTIONS "-x assembler-with-cpp") + set (SRCS_CONTEXT + ${LIBRARY_DIR}/libs/context/src/dummy.cpp + ${LIBRARY_DIR}/libs/context/src/execution_context.cpp + ${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp + ) + if (SANITIZE AND (SANITIZE STREQUAL "address" OR SANITIZE STREQUAL "thread")) add_compile_definitions(BOOST_USE_UCONTEXT) @@ -169,39 +175,34 @@ if (NOT EXTERNAL_BOOST_FOUND) add_compile_definitions(BOOST_USE_TSAN) endif() - set (SRCS_CONTEXT + set (SRCS_CONTEXT ${SRCS_CONTEXT} ${LIBRARY_DIR}/libs/context/src/fiber.cpp ${LIBRARY_DIR}/libs/context/src/continuation.cpp - ${LIBRARY_DIR}/libs/context/src/dummy.cpp - ${LIBRARY_DIR}/libs/context/src/execution_context.cpp - ${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp ) - elseif (ARCH_ARM) - set (SRCS_CONTEXT + endif() + if (ARCH_ARM) + set (SRCS_CONTEXT ${SRCS_CONTEXT} ${LIBRARY_DIR}/libs/context/src/asm/jump_arm64_aapcs_elf_gas.S ${LIBRARY_DIR}/libs/context/src/asm/make_arm64_aapcs_elf_gas.S ${LIBRARY_DIR}/libs/context/src/asm/ontop_arm64_aapcs_elf_gas.S - ${LIBRARY_DIR}/libs/context/src/dummy.cpp - ${LIBRARY_DIR}/libs/context/src/execution_context.cpp - ${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp + ) + elseif (ARCH_PPC64LE) + set (SRCS_CONTEXT ${SRCS_CONTEXT} + ${LIBRARY_DIR}/libs/context/src/asm/jump_ppc64_sysv_elf_gas.S + ${LIBRARY_DIR}/libs/context/src/asm/make_ppc64_sysv_elf_gas.S + ${LIBRARY_DIR}/libs/context/src/asm/ontop_ppc64_sysv_elf_gas.S ) elseif(OS_DARWIN) - set (SRCS_CONTEXT + set (SRCS_CONTEXT ${SRCS_CONTEXT} ${LIBRARY_DIR}/libs/context/src/asm/jump_x86_64_sysv_macho_gas.S ${LIBRARY_DIR}/libs/context/src/asm/make_x86_64_sysv_macho_gas.S ${LIBRARY_DIR}/libs/context/src/asm/ontop_x86_64_sysv_macho_gas.S - ${LIBRARY_DIR}/libs/context/src/dummy.cpp - ${LIBRARY_DIR}/libs/context/src/execution_context.cpp - ${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp ) else() - set (SRCS_CONTEXT + set (SRCS_CONTEXT ${SRCS_CONTEXT} ${LIBRARY_DIR}/libs/context/src/asm/jump_x86_64_sysv_elf_gas.S ${LIBRARY_DIR}/libs/context/src/asm/make_x86_64_sysv_elf_gas.S ${LIBRARY_DIR}/libs/context/src/asm/ontop_x86_64_sysv_elf_gas.S - ${LIBRARY_DIR}/libs/context/src/dummy.cpp - ${LIBRARY_DIR}/libs/context/src/execution_context.cpp - ${LIBRARY_DIR}/libs/context/src/posix/stack_traits.cpp ) endif() From db2e9c870f012bceb9a98a303889c554398daad9 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 2 Apr 2021 00:02:35 +0300 Subject: [PATCH 216/266] cleanup --- docker/test/fasttest/run.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c711a253e23..46ba0d7c242 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,7 +127,7 @@ continue function clone_root { - git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" + git clone https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( cd "$FASTTEST_SOURCE" @@ -222,7 +222,7 @@ function run_cmake ( cd "$FASTTEST_BUILD" - cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" + cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" ) } @@ -230,7 +230,7 @@ function build { ( cd "$FASTTEST_BUILD" - time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" + time ninja clickhouse-bundle 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" if [ "$COPY_CLICKHOUSE_BINARY_TO_OUTPUT" -eq "1" ]; then cp programs/clickhouse "$FASTTEST_OUTPUT/clickhouse" fi @@ -427,7 +427,7 @@ case "$stage" in # See the compatibility hacks in `clone_root` stage above. Remove at the same time, # after Nov 1, 2020. cd "$FASTTEST_WORKSPACE" - clone_submodules | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" + clone_submodules 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" ;& "run_cmake") run_cmake @@ -438,7 +438,7 @@ case "$stage" in "configure") # The `install_log.txt` is also needed for compatibility with old CI task -- # if there is no log, it will decide that build failed. - configure | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt" + configure 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt" ;& "run_tests") run_tests From b9fe91fcff30c3985fa2b736ffa0ae2aadc7c1c2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 2 Apr 2021 00:04:59 +0300 Subject: [PATCH 217/266] fixup --- docker/test/fasttest/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 46ba0d7c242..c21a115289d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,7 +127,7 @@ continue function clone_root { - git clone https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" + git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( cd "$FASTTEST_SOURCE" @@ -222,7 +222,7 @@ function run_cmake ( cd "$FASTTEST_BUILD" - cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" + cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" ) } From 0692459f9fe011cc22bb3e26658462fb9d3d2058 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Fri, 2 Apr 2021 00:06:21 +0300 Subject: [PATCH 218/266] ppc64le: Disable incompatible libraries (platform specific typically) --- cmake/find/base64.cmake | 6 +++++- cmake/find/fastops.cmake | 2 +- cmake/find/hdfs3.cmake | 2 +- contrib/libcpuid-cmake/CMakeLists.txt | 2 +- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/cmake/find/base64.cmake b/cmake/find/base64.cmake index 7427baf9cad..1ac1d150a89 100644 --- a/cmake/find/base64.cmake +++ b/cmake/find/base64.cmake @@ -1,4 +1,8 @@ -option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) +if(NOT ARCH_PPC64LE) + option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) +elseif(ENABLE_BASE64) + message (${RECONFIGURE_MESSAGE_LEVEL} "base64 library is not supported on PowerPC") +endif() if (NOT ENABLE_BASE64) return() diff --git a/cmake/find/fastops.cmake b/cmake/find/fastops.cmake index 5ab320bdb7a..712ddc4a2c5 100644 --- a/cmake/find/fastops.cmake +++ b/cmake/find/fastops.cmake @@ -1,4 +1,4 @@ -if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT OS_DARWIN) +if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT OS_DARWIN AND NOT ARCH_PPC64LE) option(ENABLE_FASTOPS "Enable fast vectorized mathematical functions library by Mikhail Parakhin" ${ENABLE_LIBRARIES}) elseif(ENABLE_FASTOPS) message (${RECONFIGURE_MESSAGE_LEVEL} "Fastops library is not supported on ARM, FreeBSD and Darwin") diff --git a/cmake/find/hdfs3.cmake b/cmake/find/hdfs3.cmake index 7b385f24e1e..3aab2b612ef 100644 --- a/cmake/find/hdfs3.cmake +++ b/cmake/find/hdfs3.cmake @@ -1,4 +1,4 @@ -if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT APPLE AND USE_PROTOBUF) +if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT APPLE AND USE_PROTOBUF AND NOT ARCH_PPC64LE) option(ENABLE_HDFS "Enable HDFS" ${ENABLE_LIBRARIES}) elseif(ENABLE_HDFS OR USE_INTERNAL_HDFS3_LIBRARY) message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use HDFS3 with current configuration") diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index 8c1be50b4e6..952d0855a46 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -if (NOT ARCH_ARM) +if (NOT ARCH_ARM AND NOT ARCH_PPC64LE) option (ENABLE_CPUID "Enable libcpuid library (only internal)" ${ENABLE_LIBRARIES}) endif() From 6d0349aec60af8b7c5046878d7960d88c67d8ee4 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Fri, 2 Apr 2021 00:15:31 +0300 Subject: [PATCH 219/266] ppc64le: Re-enable compiling with Clang --- cmake/tools.cmake | 5 ----- 1 file changed, 5 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index abb11843d59..44fc3b3e530 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -86,8 +86,3 @@ if (LINKER_NAME) message(STATUS "Using custom linker by name: ${LINKER_NAME}") endif () -if (ARCH_PPC64LE) - if (COMPILER_CLANG OR (COMPILER_GCC AND CMAKE_CXX_COMPILER_VERSION VERSION_LESS 8)) - message(FATAL_ERROR "Only gcc-8 or higher is supported for powerpc architecture") - endif () -endif () From b5ecfaa3c2b2e4cc886f61723434829c631b1919 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Apr 2021 01:41:28 +0300 Subject: [PATCH 220/266] Remove TestFlows (2) --- tests/testflows/regression.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index 05fec3ea985..45f1ed64a6c 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -14,10 +14,10 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): """ args = {"local": local, "clickhouse_binary_path": clickhouse_binary_path, "stress": stress, "parallel": parallel} - Feature(test=load("example.regression", "regression"))(**args) - Feature(test=load("ldap.regression", "regression"))(**args) - Feature(test=load("rbac.regression", "regression"))(**args) - Feature(test=load("aes_encryption.regression", "regression"))(**args) + # Feature(test=load("example.regression", "regression"))(**args) + # Feature(test=load("ldap.regression", "regression"))(**args) + # Feature(test=load("rbac.regression", "regression"))(**args) + # Feature(test=load("aes_encryption.regression", "regression"))(**args) # Feature(test=load("kerberos.regression", "regression"))(**args) if main(): From 952429b73e476a0a063ae46467df85056e0f5851 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 2 Apr 2021 05:26:56 +0300 Subject: [PATCH 221/266] Update 01781_token_extractor_buffer_overflow.sql --- .../0_stateless/01781_token_extractor_buffer_overflow.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql index 00be1f46aba..4cc216955b3 100644 --- a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql @@ -1,4 +1,4 @@ -SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20; +SET max_block_size = 10, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20; DROP TABLE IF EXISTS bloom_filter; CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8; From 6be8746d8cb716ca39b4aab7845277532d10fb50 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Fri, 2 Apr 2021 05:48:23 +0300 Subject: [PATCH 222/266] aarch64: Enable s3 --- cmake/find/s3.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/find/s3.cmake b/cmake/find/s3.cmake index 1bbf48fd6b0..1b0c652a31a 100644 --- a/cmake/find/s3.cmake +++ b/cmake/find/s3.cmake @@ -1,7 +1,7 @@ -if(NOT OS_FREEBSD AND NOT APPLE AND NOT ARCH_ARM) +if(NOT OS_FREEBSD AND NOT APPLE) option(ENABLE_S3 "Enable S3" ${ENABLE_LIBRARIES}) elseif(ENABLE_S3 OR USE_INTERNAL_AWS_S3_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on ARM, Apple or FreeBSD") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use S3 on Apple or FreeBSD") endif() if(NOT ENABLE_S3) From b4003a07d2d726d5ad2bb9423b48c9d10a6e5589 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Fri, 2 Apr 2021 06:32:09 +0300 Subject: [PATCH 223/266] ppc64le: Enable the bundled openldap --- cmake/find/ldap.cmake | 1 + .../linux_ppc64le/include/lber_types.h | 63 + .../linux_ppc64le/include/ldap_config.h | 74 ++ .../linux_ppc64le/include/ldap_features.h | 61 + .../linux_ppc64le/include/portable.h | 1169 +++++++++++++++++ 5 files changed, 1368 insertions(+) create mode 100644 contrib/openldap-cmake/linux_ppc64le/include/lber_types.h create mode 100644 contrib/openldap-cmake/linux_ppc64le/include/ldap_config.h create mode 100644 contrib/openldap-cmake/linux_ppc64le/include/ldap_features.h create mode 100644 contrib/openldap-cmake/linux_ppc64le/include/portable.h diff --git a/cmake/find/ldap.cmake b/cmake/find/ldap.cmake index 369c1e42e8d..0dffa334e73 100644 --- a/cmake/find/ldap.cmake +++ b/cmake/find/ldap.cmake @@ -62,6 +62,7 @@ if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) if ( ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "x86_64" ) OR ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR + ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "ppc64le" ) OR ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) ) diff --git a/contrib/openldap-cmake/linux_ppc64le/include/lber_types.h b/contrib/openldap-cmake/linux_ppc64le/include/lber_types.h new file mode 100644 index 00000000000..dbd59430527 --- /dev/null +++ b/contrib/openldap-cmake/linux_ppc64le/include/lber_types.h @@ -0,0 +1,63 @@ +/* include/lber_types.h. Generated from lber_types.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * LBER types + */ + +#ifndef _LBER_TYPES_H +#define _LBER_TYPES_H + +#include + +LDAP_BEGIN_DECL + +/* LBER boolean, enum, integers (32 bits or larger) */ +#define LBER_INT_T int + +/* LBER tags (32 bits or larger) */ +#define LBER_TAG_T long + +/* LBER socket descriptor */ +#define LBER_SOCKET_T int + +/* LBER lengths (32 bits or larger) */ +#define LBER_LEN_T long + +/* ------------------------------------------------------------ */ + +/* booleans, enumerations, and integers */ +typedef LBER_INT_T ber_int_t; + +/* signed and unsigned versions */ +typedef signed LBER_INT_T ber_sint_t; +typedef unsigned LBER_INT_T ber_uint_t; + +/* tags */ +typedef unsigned LBER_TAG_T ber_tag_t; + +/* "socket" descriptors */ +typedef LBER_SOCKET_T ber_socket_t; + +/* lengths */ +typedef unsigned LBER_LEN_T ber_len_t; + +/* signed lengths */ +typedef signed LBER_LEN_T ber_slen_t; + +LDAP_END_DECL + +#endif /* _LBER_TYPES_H */ diff --git a/contrib/openldap-cmake/linux_ppc64le/include/ldap_config.h b/contrib/openldap-cmake/linux_ppc64le/include/ldap_config.h new file mode 100644 index 00000000000..89f7b40b884 --- /dev/null +++ b/contrib/openldap-cmake/linux_ppc64le/include/ldap_config.h @@ -0,0 +1,74 @@ +/* include/ldap_config.h. Generated from ldap_config.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * This file works in conjunction with OpenLDAP configure system. + * If you do no like the values below, adjust your configure options. + */ + +#ifndef _LDAP_CONFIG_H +#define _LDAP_CONFIG_H + +/* directory separator */ +#ifndef LDAP_DIRSEP +#ifndef _WIN32 +#define LDAP_DIRSEP "/" +#else +#define LDAP_DIRSEP "\\" +#endif +#endif + +/* directory for temporary files */ +#if defined(_WIN32) +# define LDAP_TMPDIR "C:\\." /* we don't have much of a choice */ +#elif defined( _P_tmpdir ) +# define LDAP_TMPDIR _P_tmpdir +#elif defined( P_tmpdir ) +# define LDAP_TMPDIR P_tmpdir +#elif defined( _PATH_TMPDIR ) +# define LDAP_TMPDIR _PATH_TMPDIR +#else +# define LDAP_TMPDIR LDAP_DIRSEP "tmp" +#endif + +/* directories */ +#ifndef LDAP_BINDIR +#define LDAP_BINDIR "/tmp/ldap-prefix/bin" +#endif +#ifndef LDAP_SBINDIR +#define LDAP_SBINDIR "/tmp/ldap-prefix/sbin" +#endif +#ifndef LDAP_DATADIR +#define LDAP_DATADIR "/tmp/ldap-prefix/share/openldap" +#endif +#ifndef LDAP_SYSCONFDIR +#define LDAP_SYSCONFDIR "/tmp/ldap-prefix/etc/openldap" +#endif +#ifndef LDAP_LIBEXECDIR +#define LDAP_LIBEXECDIR "/tmp/ldap-prefix/libexec" +#endif +#ifndef LDAP_MODULEDIR +#define LDAP_MODULEDIR "/tmp/ldap-prefix/libexec/openldap" +#endif +#ifndef LDAP_RUNDIR +#define LDAP_RUNDIR "/tmp/ldap-prefix/var" +#endif +#ifndef LDAP_LOCALEDIR +#define LDAP_LOCALEDIR "" +#endif + + +#endif /* _LDAP_CONFIG_H */ diff --git a/contrib/openldap-cmake/linux_ppc64le/include/ldap_features.h b/contrib/openldap-cmake/linux_ppc64le/include/ldap_features.h new file mode 100644 index 00000000000..f0cc7c3626f --- /dev/null +++ b/contrib/openldap-cmake/linux_ppc64le/include/ldap_features.h @@ -0,0 +1,61 @@ +/* include/ldap_features.h. Generated from ldap_features.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * LDAP Features + */ + +#ifndef _LDAP_FEATURES_H +#define _LDAP_FEATURES_H 1 + +/* OpenLDAP API version macros */ +#define LDAP_VENDOR_VERSION 20501 +#define LDAP_VENDOR_VERSION_MAJOR 2 +#define LDAP_VENDOR_VERSION_MINOR 5 +#define LDAP_VENDOR_VERSION_PATCH X + +/* +** WORK IN PROGRESS! +** +** OpenLDAP reentrancy/thread-safeness should be dynamically +** checked using ldap_get_option(). +** +** The -lldap implementation is not thread-safe. +** +** The -lldap_r implementation is: +** LDAP_API_FEATURE_THREAD_SAFE (basic thread safety) +** but also be: +** LDAP_API_FEATURE_SESSION_THREAD_SAFE +** LDAP_API_FEATURE_OPERATION_THREAD_SAFE +** +** The preprocessor flag LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE +** can be used to determine if -lldap_r is available at compile +** time. You must define LDAP_THREAD_SAFE if and only if you +** link with -lldap_r. +** +** If you fail to define LDAP_THREAD_SAFE when linking with +** -lldap_r or define LDAP_THREAD_SAFE when linking with -lldap, +** provided header definitions and declarations may be incorrect. +** +*/ + +/* is -lldap_r available or not */ +#define LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE 1 + +/* LDAP v2 Referrals */ +/* #undef LDAP_API_FEATURE_X_OPENLDAP_V2_REFERRALS */ + +#endif /* LDAP_FEATURES */ diff --git a/contrib/openldap-cmake/linux_ppc64le/include/portable.h b/contrib/openldap-cmake/linux_ppc64le/include/portable.h new file mode 100644 index 00000000000..2924b6713a4 --- /dev/null +++ b/contrib/openldap-cmake/linux_ppc64le/include/portable.h @@ -0,0 +1,1169 @@ +/* include/portable.h. Generated from portable.hin by configure. */ +/* include/portable.hin. Generated from configure.in by autoheader. */ + + +/* begin of portable.h.pre */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in the file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +#ifndef _LDAP_PORTABLE_H +#define _LDAP_PORTABLE_H + +/* define this if needed to get reentrant functions */ +#ifndef REENTRANT +#define REENTRANT 1 +#endif +#ifndef _REENTRANT +#define _REENTRANT 1 +#endif + +/* define this if needed to get threadsafe functions */ +#ifndef THREADSAFE +#define THREADSAFE 1 +#endif +#ifndef _THREADSAFE +#define _THREADSAFE 1 +#endif +#ifndef THREAD_SAFE +#define THREAD_SAFE 1 +#endif +#ifndef _THREAD_SAFE +#define _THREAD_SAFE 1 +#endif + +#ifndef _SGI_MP_SOURCE +#define _SGI_MP_SOURCE 1 +#endif + +/* end of portable.h.pre */ + + +/* Define if building universal (internal helper macro) */ +/* #undef AC_APPLE_UNIVERSAL_BUILD */ + +/* define to use both and */ +/* #undef BOTH_STRINGS_H */ + +/* define if cross compiling */ +/* #undef CROSS_COMPILING */ + +/* set to the number of arguments ctime_r() expects */ +#define CTIME_R_NARGS 2 + +/* define if toupper() requires islower() */ +/* #undef C_UPPER_LOWER */ + +/* define if sys_errlist is not declared in stdio.h or errno.h */ +/* #undef DECL_SYS_ERRLIST */ + +/* define to enable slapi library */ +/* #undef ENABLE_SLAPI */ + +/* defined to be the EXE extension */ +#define EXEEXT "" + +/* set to the number of arguments gethostbyaddr_r() expects */ +#define GETHOSTBYADDR_R_NARGS 8 + +/* set to the number of arguments gethostbyname_r() expects */ +#define GETHOSTBYNAME_R_NARGS 6 + +/* Define to 1 if `TIOCGWINSZ' requires . */ +#define GWINSZ_IN_SYS_IOCTL 1 + +/* define if you have AIX security lib */ +/* #undef HAVE_AIX_SECURITY */ + +/* Define to 1 if you have the header file. */ +#define HAVE_ARPA_INET_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ARPA_NAMESER_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ASSERT_H 1 + +/* Define to 1 if you have the `bcopy' function. */ +#define HAVE_BCOPY 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_BITS_TYPES_H 1 + +/* Define to 1 if you have the `chroot' function. */ +#define HAVE_CHROOT 1 + +/* Define to 1 if you have the `closesocket' function. */ +/* #undef HAVE_CLOSESOCKET */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_CONIO_H */ + +/* define if crypt(3) is available */ +/* #undef HAVE_CRYPT */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_CRYPT_H */ + +/* define if crypt_r() is also available */ +/* #undef HAVE_CRYPT_R */ + +/* Define to 1 if you have the `ctime_r' function. */ +#define HAVE_CTIME_R 1 + +/* define if you have Cyrus SASL */ +/* #undef HAVE_CYRUS_SASL */ + +/* define if your system supports /dev/poll */ +/* #undef HAVE_DEVPOLL */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_DIRECT_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +#define HAVE_DIRENT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you don't have `vprintf' but do have `_doprnt.' */ +/* #undef HAVE_DOPRNT */ + +/* define if system uses EBCDIC instead of ASCII */ +/* #undef HAVE_EBCDIC */ + +/* Define to 1 if you have the `endgrent' function. */ +#define HAVE_ENDGRENT 1 + +/* Define to 1 if you have the `endpwent' function. */ +#define HAVE_ENDPWENT 1 + +/* define if your system supports epoll */ +#define HAVE_EPOLL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ERRNO_H 1 + +/* Define to 1 if you have the `fcntl' function. */ +#define HAVE_FCNTL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_FCNTL_H 1 + +/* define if you actually have FreeBSD fetch(3) */ +/* #undef HAVE_FETCH */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_FILIO_H */ + +/* Define to 1 if you have the `flock' function. */ +#define HAVE_FLOCK 1 + +/* Define to 1 if you have the `fstat' function. */ +#define HAVE_FSTAT 1 + +/* Define to 1 if you have the `gai_strerror' function. */ +#define HAVE_GAI_STRERROR 1 + +/* Define to 1 if you have the `getaddrinfo' function. */ +#define HAVE_GETADDRINFO 1 + +/* Define to 1 if you have the `getdtablesize' function. */ +#define HAVE_GETDTABLESIZE 1 + +/* Define to 1 if you have the `geteuid' function. */ +#define HAVE_GETEUID 1 + +/* Define to 1 if you have the `getgrgid' function. */ +#define HAVE_GETGRGID 1 + +/* Define to 1 if you have the `gethostbyaddr_r' function. */ +#define HAVE_GETHOSTBYADDR_R 1 + +/* Define to 1 if you have the `gethostbyname_r' function. */ +#define HAVE_GETHOSTBYNAME_R 1 + +/* Define to 1 if you have the `gethostname' function. */ +#define HAVE_GETHOSTNAME 1 + +/* Define to 1 if you have the `getnameinfo' function. */ +#define HAVE_GETNAMEINFO 1 + +/* Define to 1 if you have the `getopt' function. */ +#define HAVE_GETOPT 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_GETOPT_H 1 + +/* Define to 1 if you have the `getpassphrase' function. */ +/* #undef HAVE_GETPASSPHRASE */ + +/* Define to 1 if you have the `getpeereid' function. */ +/* #undef HAVE_GETPEEREID */ + +/* Define to 1 if you have the `getpeerucred' function. */ +/* #undef HAVE_GETPEERUCRED */ + +/* Define to 1 if you have the `getpwnam' function. */ +#define HAVE_GETPWNAM 1 + +/* Define to 1 if you have the `getpwuid' function. */ +#define HAVE_GETPWUID 1 + +/* Define to 1 if you have the `getspnam' function. */ +#define HAVE_GETSPNAM 1 + +/* Define to 1 if you have the `gettimeofday' function. */ +#define HAVE_GETTIMEOFDAY 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GMP_H */ + +/* Define to 1 if you have the `gmtime_r' function. */ +#define HAVE_GMTIME_R 1 + +/* define if you have GNUtls */ +/* #undef HAVE_GNUTLS */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GNUTLS_GNUTLS_H */ + +/* if you have GNU Pth */ +/* #undef HAVE_GNU_PTH */ + +/* Define to 1 if you have the header file. */ +#define HAVE_GRP_H 1 + +/* Define to 1 if you have the `hstrerror' function. */ +#define HAVE_HSTRERROR 1 + +/* define to you inet_aton(3) is available */ +#define HAVE_INET_ATON 1 + +/* Define to 1 if you have the `inet_ntoa_b' function. */ +/* #undef HAVE_INET_NTOA_B */ + +/* Define to 1 if you have the `inet_ntop' function. */ +#define HAVE_INET_NTOP 1 + +/* Define to 1 if you have the `initgroups' function. */ +#define HAVE_INITGROUPS 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the `ioctl' function. */ +#define HAVE_IOCTL 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_IO_H */ + +/* define if your system supports kqueue */ +/* #undef HAVE_KQUEUE */ + +/* Define to 1 if you have the `gen' library (-lgen). */ +/* #undef HAVE_LIBGEN */ + +/* Define to 1 if you have the `gmp' library (-lgmp). */ +/* #undef HAVE_LIBGMP */ + +/* Define to 1 if you have the `inet' library (-linet). */ +/* #undef HAVE_LIBINET */ + +/* define if you have libtool -ltdl */ +/* #undef HAVE_LIBLTDL */ + +/* Define to 1 if you have the `net' library (-lnet). */ +/* #undef HAVE_LIBNET */ + +/* Define to 1 if you have the `nsl' library (-lnsl). */ +/* #undef HAVE_LIBNSL */ + +/* Define to 1 if you have the `nsl_s' library (-lnsl_s). */ +/* #undef HAVE_LIBNSL_S */ + +/* Define to 1 if you have the `socket' library (-lsocket). */ +/* #undef HAVE_LIBSOCKET */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LIBUTIL_H */ + +/* Define to 1 if you have the `V3' library (-lV3). */ +/* #undef HAVE_LIBV3 */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LIMITS_H 1 + +/* if you have LinuxThreads */ +/* #undef HAVE_LINUX_THREADS */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LOCALE_H 1 + +/* Define to 1 if you have the `localtime_r' function. */ +#define HAVE_LOCALTIME_R 1 + +/* Define to 1 if you have the `lockf' function. */ +#define HAVE_LOCKF 1 + +/* Define to 1 if the system has the type `long long'. */ +#define HAVE_LONG_LONG 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LTDL_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_MALLOC_H 1 + +/* Define to 1 if you have the `memcpy' function. */ +#define HAVE_MEMCPY 1 + +/* Define to 1 if you have the `memmove' function. */ +#define HAVE_MEMMOVE 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_MEMORY_H 1 + +/* Define to 1 if you have the `memrchr' function. */ +#define HAVE_MEMRCHR 1 + +/* Define to 1 if you have the `mkstemp' function. */ +#define HAVE_MKSTEMP 1 + +/* Define to 1 if you have the `mktemp' function. */ +#define HAVE_MKTEMP 1 + +/* define this if you have mkversion */ +#define HAVE_MKVERSION 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. */ +/* #undef HAVE_NDIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_NETINET_TCP_H 1 + +/* define if strerror_r returns char* instead of int */ +/* #undef HAVE_NONPOSIX_STRERROR_R */ + +/* if you have NT Event Log */ +/* #undef HAVE_NT_EVENT_LOG */ + +/* if you have NT Service Manager */ +/* #undef HAVE_NT_SERVICE_MANAGER */ + +/* if you have NT Threads */ +/* #undef HAVE_NT_THREADS */ + +/* define if you have OpenSSL */ +#define HAVE_OPENSSL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_BN_H 1 + +/* define if you have OpenSSL with CRL checking capability */ +#define HAVE_OPENSSL_CRL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_CRYPTO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_SSL_H 1 + +/* Define to 1 if you have the `pipe' function. */ +#define HAVE_PIPE 1 + +/* Define to 1 if you have the `poll' function. */ +#define HAVE_POLL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PROCESS_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PSAP_H */ + +/* define to pthreads API spec revision */ +#define HAVE_PTHREADS 10 + +/* define if you have pthread_detach function */ +#define HAVE_PTHREAD_DETACH 1 + +/* Define to 1 if you have the `pthread_getconcurrency' function. */ +#define HAVE_PTHREAD_GETCONCURRENCY 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PTHREAD_H 1 + +/* Define to 1 if you have the `pthread_kill' function. */ +#define HAVE_PTHREAD_KILL 1 + +/* Define to 1 if you have the `pthread_kill_other_threads_np' function. */ +/* #undef HAVE_PTHREAD_KILL_OTHER_THREADS_NP */ + +/* define if you have pthread_rwlock_destroy function */ +#define HAVE_PTHREAD_RWLOCK_DESTROY 1 + +/* Define to 1 if you have the `pthread_setconcurrency' function. */ +#define HAVE_PTHREAD_SETCONCURRENCY 1 + +/* Define to 1 if you have the `pthread_yield' function. */ +#define HAVE_PTHREAD_YIELD 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PTH_H */ + +/* Define to 1 if the system has the type `ptrdiff_t'. */ +#define HAVE_PTRDIFF_T 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PWD_H 1 + +/* Define to 1 if you have the `read' function. */ +#define HAVE_READ 1 + +/* Define to 1 if you have the `recv' function. */ +#define HAVE_RECV 1 + +/* Define to 1 if you have the `recvfrom' function. */ +#define HAVE_RECVFROM 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_REGEX_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_RESOLV_H */ + +/* define if you have res_query() */ +/* #undef HAVE_RES_QUERY */ + +/* define if OpenSSL needs RSAref */ +/* #undef HAVE_RSAREF */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SASL_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SASL_SASL_H */ + +/* define if your SASL library has sasl_version() */ +/* #undef HAVE_SASL_VERSION */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SCHED_H 1 + +/* Define to 1 if you have the `sched_yield' function. */ +#define HAVE_SCHED_YIELD 1 + +/* Define to 1 if you have the `send' function. */ +#define HAVE_SEND 1 + +/* Define to 1 if you have the `sendmsg' function. */ +#define HAVE_SENDMSG 1 + +/* Define to 1 if you have the `sendto' function. */ +#define HAVE_SENDTO 1 + +/* Define to 1 if you have the `setegid' function. */ +#define HAVE_SETEGID 1 + +/* Define to 1 if you have the `seteuid' function. */ +#define HAVE_SETEUID 1 + +/* Define to 1 if you have the `setgid' function. */ +#define HAVE_SETGID 1 + +/* Define to 1 if you have the `setpwfile' function. */ +/* #undef HAVE_SETPWFILE */ + +/* Define to 1 if you have the `setsid' function. */ +#define HAVE_SETSID 1 + +/* Define to 1 if you have the `setuid' function. */ +#define HAVE_SETUID 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SGTTY_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SHADOW_H */ + +/* Define to 1 if you have the `sigaction' function. */ +#define HAVE_SIGACTION 1 + +/* Define to 1 if you have the `signal' function. */ +#define HAVE_SIGNAL 1 + +/* Define to 1 if you have the `sigset' function. */ +#define HAVE_SIGSET 1 + +/* define if you have -lslp */ +/* #undef HAVE_SLP */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SLP_H */ + +/* Define to 1 if you have the `snprintf' function. */ +#define HAVE_SNPRINTF 1 + +/* if you have spawnlp() */ +/* #undef HAVE_SPAWNLP */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SQLEXT_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SQL_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_STDDEF_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the `strdup' function. */ +#define HAVE_STRDUP 1 + +/* Define to 1 if you have the `strerror' function. */ +#define HAVE_STRERROR 1 + +/* Define to 1 if you have the `strerror_r' function. */ +#define HAVE_STRERROR_R 1 + +/* Define to 1 if you have the `strftime' function. */ +#define HAVE_STRFTIME 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the `strpbrk' function. */ +#define HAVE_STRPBRK 1 + +/* Define to 1 if you have the `strrchr' function. */ +#define HAVE_STRRCHR 1 + +/* Define to 1 if you have the `strsep' function. */ +#define HAVE_STRSEP 1 + +/* Define to 1 if you have the `strspn' function. */ +#define HAVE_STRSPN 1 + +/* Define to 1 if you have the `strstr' function. */ +#define HAVE_STRSTR 1 + +/* Define to 1 if you have the `strtol' function. */ +#define HAVE_STRTOL 1 + +/* Define to 1 if you have the `strtoll' function. */ +#define HAVE_STRTOLL 1 + +/* Define to 1 if you have the `strtoq' function. */ +#define HAVE_STRTOQ 1 + +/* Define to 1 if you have the `strtoul' function. */ +#define HAVE_STRTOUL 1 + +/* Define to 1 if you have the `strtoull' function. */ +#define HAVE_STRTOULL 1 + +/* Define to 1 if you have the `strtouq' function. */ +#define HAVE_STRTOUQ 1 + +/* Define to 1 if `msg_accrightslen' is a member of `struct msghdr'. */ +/* #undef HAVE_STRUCT_MSGHDR_MSG_ACCRIGHTSLEN */ + +/* Define to 1 if `msg_control' is a member of `struct msghdr'. */ +#define HAVE_STRUCT_MSGHDR_MSG_CONTROL 1 + +/* Define to 1 if `pw_gecos' is a member of `struct passwd'. */ +#define HAVE_STRUCT_PASSWD_PW_GECOS 1 + +/* Define to 1 if `pw_passwd' is a member of `struct passwd'. */ +#define HAVE_STRUCT_PASSWD_PW_PASSWD 1 + +/* Define to 1 if `st_blksize' is a member of `struct stat'. */ +#define HAVE_STRUCT_STAT_ST_BLKSIZE 1 + +/* Define to 1 if `st_fstype' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE */ + +/* define to 1 if st_fstype is char * */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE_CHAR */ + +/* define to 1 if st_fstype is int */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE_INT */ + +/* Define to 1 if `st_vfstype' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_VFSTYPE */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYNCH_H */ + +/* Define to 1 if you have the `sysconf' function. */ +#define HAVE_SYSCONF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYSEXITS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYSLOG_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_DEVPOLL_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_DIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_EPOLL_H 1 + +/* define if you actually have sys_errlist in your libs */ +#define HAVE_SYS_ERRLIST 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_ERRNO_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_EVENT_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_FILE_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_FILIO_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_FSTYP_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_IOCTL_H 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_NDIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_PARAM_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_PRIVGRP_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_RESOURCE_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SELECT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SOCKET_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SYSLOG_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TIME_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_UCRED_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UIO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UN_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_UUID_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_VMOUNT_H */ + +/* Define to 1 if you have that is POSIX.1 compatible. */ +#define HAVE_SYS_WAIT_H 1 + +/* define if you have -lwrap */ +/* #undef HAVE_TCPD */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_TCPD_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_TERMIOS_H 1 + +/* if you have Solaris LWP (thr) package */ +/* #undef HAVE_THR */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_THREAD_H */ + +/* Define to 1 if you have the `thr_getconcurrency' function. */ +/* #undef HAVE_THR_GETCONCURRENCY */ + +/* Define to 1 if you have the `thr_setconcurrency' function. */ +/* #undef HAVE_THR_SETCONCURRENCY */ + +/* Define to 1 if you have the `thr_yield' function. */ +/* #undef HAVE_THR_YIELD */ + +/* define if you have TLS */ +#define HAVE_TLS 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UTIME_H 1 + +/* define if you have uuid_generate() */ +/* #undef HAVE_UUID_GENERATE */ + +/* define if you have uuid_to_str() */ +/* #undef HAVE_UUID_TO_STR */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_UUID_UUID_H */ + +/* Define to 1 if you have the `vprintf' function. */ +#define HAVE_VPRINTF 1 + +/* Define to 1 if you have the `vsnprintf' function. */ +#define HAVE_VSNPRINTF 1 + +/* Define to 1 if you have the `wait4' function. */ +#define HAVE_WAIT4 1 + +/* Define to 1 if you have the `waitpid' function. */ +#define HAVE_WAITPID 1 + +/* define if you have winsock */ +/* #undef HAVE_WINSOCK */ + +/* define if you have winsock2 */ +/* #undef HAVE_WINSOCK2 */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINSOCK2_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINSOCK_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WIREDTIGER_H */ + +/* Define to 1 if you have the `write' function. */ +#define HAVE_WRITE 1 + +/* define if select implicitly yields */ +#define HAVE_YIELDING_SELECT 1 + +/* Define to 1 if you have the `_vsnprintf' function. */ +/* #undef HAVE__VSNPRINTF */ + +/* define to 32-bit or greater integer type */ +#define LBER_INT_T int + +/* define to large integer type */ +#define LBER_LEN_T long + +/* define to socket descriptor type */ +#define LBER_SOCKET_T int + +/* define to large integer type */ +#define LBER_TAG_T long + +/* define to 1 if library is thread safe */ +#define LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE 1 + +/* define to LDAP VENDOR VERSION */ +/* #undef LDAP_API_FEATURE_X_OPENLDAP_V2_REFERRALS */ + +/* define this to add debugging code */ +/* #undef LDAP_DEBUG */ + +/* define if LDAP libs are dynamic */ +/* #undef LDAP_LIBS_DYNAMIC */ + +/* define to support PF_INET6 */ +#define LDAP_PF_INET6 1 + +/* define to support PF_LOCAL */ +#define LDAP_PF_LOCAL 1 + +/* define this to add SLAPI code */ +/* #undef LDAP_SLAPI */ + +/* define this to add syslog code */ +/* #undef LDAP_SYSLOG */ + +/* Version */ +#define LDAP_VENDOR_VERSION 20501 + +/* Major */ +#define LDAP_VENDOR_VERSION_MAJOR 2 + +/* Minor */ +#define LDAP_VENDOR_VERSION_MINOR 5 + +/* Patch */ +#define LDAP_VENDOR_VERSION_PATCH X + +/* Define to the sub-directory where libtool stores uninstalled libraries. */ +#define LT_OBJDIR ".libs/" + +/* define if memcmp is not 8-bit clean or is otherwise broken */ +/* #undef NEED_MEMCMP_REPLACEMENT */ + +/* define if you have (or want) no threads */ +/* #undef NO_THREADS */ + +/* define to use the original debug style */ +/* #undef OLD_DEBUG */ + +/* Package */ +#define OPENLDAP_PACKAGE "OpenLDAP" + +/* Version */ +#define OPENLDAP_VERSION "2.5.X" + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "" + +/* define if sched_yield yields the entire process */ +/* #undef REPLACE_BROKEN_YIELD */ + +/* Define as the return type of signal handlers (`int' or `void'). */ +#define RETSIGTYPE void + +/* Define to the type of arg 1 for `select'. */ +#define SELECT_TYPE_ARG1 int + +/* Define to the type of args 2, 3 and 4 for `select'. */ +#define SELECT_TYPE_ARG234 (fd_set *) + +/* Define to the type of arg 5 for `select'. */ +#define SELECT_TYPE_ARG5 (struct timeval *) + +/* The size of `int', as computed by sizeof. */ +#define SIZEOF_INT 4 + +/* The size of `long', as computed by sizeof. */ +#define SIZEOF_LONG 8 + +/* The size of `long long', as computed by sizeof. */ +#define SIZEOF_LONG_LONG 8 + +/* The size of `short', as computed by sizeof. */ +#define SIZEOF_SHORT 2 + +/* The size of `wchar_t', as computed by sizeof. */ +#define SIZEOF_WCHAR_T 4 + +/* define to support per-object ACIs */ +/* #undef SLAPD_ACI_ENABLED */ + +/* define to support LDAP Async Metadirectory backend */ +/* #undef SLAPD_ASYNCMETA */ + +/* define to support cleartext passwords */ +/* #undef SLAPD_CLEARTEXT */ + +/* define to support crypt(3) passwords */ +/* #undef SLAPD_CRYPT */ + +/* define to support DNS SRV backend */ +/* #undef SLAPD_DNSSRV */ + +/* define to support LDAP backend */ +/* #undef SLAPD_LDAP */ + +/* define to support MDB backend */ +/* #undef SLAPD_MDB */ + +/* define to support LDAP Metadirectory backend */ +/* #undef SLAPD_META */ + +/* define to support modules */ +/* #undef SLAPD_MODULES */ + +/* dynamically linked module */ +#define SLAPD_MOD_DYNAMIC 2 + +/* statically linked module */ +#define SLAPD_MOD_STATIC 1 + +/* define to support cn=Monitor backend */ +/* #undef SLAPD_MONITOR */ + +/* define to support NDB backend */ +/* #undef SLAPD_NDB */ + +/* define to support NULL backend */ +/* #undef SLAPD_NULL */ + +/* define for In-Directory Access Logging overlay */ +/* #undef SLAPD_OVER_ACCESSLOG */ + +/* define for Audit Logging overlay */ +/* #undef SLAPD_OVER_AUDITLOG */ + +/* define for Automatic Certificate Authority overlay */ +/* #undef SLAPD_OVER_AUTOCA */ + +/* define for Collect overlay */ +/* #undef SLAPD_OVER_COLLECT */ + +/* define for Attribute Constraint overlay */ +/* #undef SLAPD_OVER_CONSTRAINT */ + +/* define for Dynamic Directory Services overlay */ +/* #undef SLAPD_OVER_DDS */ + +/* define for Dynamic Directory Services overlay */ +/* #undef SLAPD_OVER_DEREF */ + +/* define for Dynamic Group overlay */ +/* #undef SLAPD_OVER_DYNGROUP */ + +/* define for Dynamic List overlay */ +/* #undef SLAPD_OVER_DYNLIST */ + +/* define for Reverse Group Membership overlay */ +/* #undef SLAPD_OVER_MEMBEROF */ + +/* define for Password Policy overlay */ +/* #undef SLAPD_OVER_PPOLICY */ + +/* define for Proxy Cache overlay */ +/* #undef SLAPD_OVER_PROXYCACHE */ + +/* define for Referential Integrity overlay */ +/* #undef SLAPD_OVER_REFINT */ + +/* define for Return Code overlay */ +/* #undef SLAPD_OVER_RETCODE */ + +/* define for Rewrite/Remap overlay */ +/* #undef SLAPD_OVER_RWM */ + +/* define for Sequential Modify overlay */ +/* #undef SLAPD_OVER_SEQMOD */ + +/* define for ServerSideSort/VLV overlay */ +/* #undef SLAPD_OVER_SSSVLV */ + +/* define for Syncrepl Provider overlay */ +/* #undef SLAPD_OVER_SYNCPROV */ + +/* define for Translucent Proxy overlay */ +/* #undef SLAPD_OVER_TRANSLUCENT */ + +/* define for Attribute Uniqueness overlay */ +/* #undef SLAPD_OVER_UNIQUE */ + +/* define for Value Sorting overlay */ +/* #undef SLAPD_OVER_VALSORT */ + +/* define to support PASSWD backend */ +/* #undef SLAPD_PASSWD */ + +/* define to support PERL backend */ +/* #undef SLAPD_PERL */ + +/* define to support relay backend */ +/* #undef SLAPD_RELAY */ + +/* define to support reverse lookups */ +/* #undef SLAPD_RLOOKUPS */ + +/* define to support SHELL backend */ +/* #undef SLAPD_SHELL */ + +/* define to support SOCK backend */ +/* #undef SLAPD_SOCK */ + +/* define to support SASL passwords */ +/* #undef SLAPD_SPASSWD */ + +/* define to support SQL backend */ +/* #undef SLAPD_SQL */ + +/* define to support WiredTiger backend */ +/* #undef SLAPD_WT */ + +/* define to support run-time loadable ACL */ +/* #undef SLAP_DYNACL */ + +/* Define to 1 if you have the ANSI C header files. */ +#define STDC_HEADERS 1 + +/* Define to 1 if you can safely include both and . */ +#define TIME_WITH_SYS_TIME 1 + +/* Define to 1 if your declares `struct tm'. */ +/* #undef TM_IN_SYS_TIME */ + +/* set to urandom device */ +#define URANDOM_DEVICE "/dev/urandom" + +/* define to use OpenSSL BIGNUM for MP */ +/* #undef USE_MP_BIGNUM */ + +/* define to use GMP for MP */ +/* #undef USE_MP_GMP */ + +/* define to use 'long' for MP */ +/* #undef USE_MP_LONG */ + +/* define to use 'long long' for MP */ +/* #undef USE_MP_LONG_LONG */ + +/* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most + significant byte first (like Motorola and SPARC, unlike Intel). */ +#if defined AC_APPLE_UNIVERSAL_BUILD +# if defined __BIG_ENDIAN__ +# define WORDS_BIGENDIAN 1 +# endif +#else +# ifndef WORDS_BIGENDIAN +/* # undef WORDS_BIGENDIAN */ +# endif +#endif + +/* Define to the type of arg 3 for `accept'. */ +#define ber_socklen_t socklen_t + +/* Define to `char *' if does not define. */ +/* #undef caddr_t */ + +/* Define to empty if `const' does not conform to ANSI C. */ +/* #undef const */ + +/* Define to `int' if doesn't define. */ +/* #undef gid_t */ + +/* Define to `int' if does not define. */ +/* #undef mode_t */ + +/* Define to `long' if does not define. */ +/* #undef off_t */ + +/* Define to `int' if does not define. */ +/* #undef pid_t */ + +/* Define to `int' if does not define. */ +/* #undef sig_atomic_t */ + +/* Define to `unsigned' if does not define. */ +/* #undef size_t */ + +/* define to snprintf routine */ +/* #undef snprintf */ + +/* Define like ber_socklen_t if does not define. */ +/* #undef socklen_t */ + +/* Define to `signed int' if does not define. */ +/* #undef ssize_t */ + +/* Define to `int' if doesn't define. */ +/* #undef uid_t */ + +/* define as empty if volatile is not supported */ +/* #undef volatile */ + +/* define to snprintf routine */ +/* #undef vsnprintf */ + + +/* begin of portable.h.post */ + +#ifdef _WIN32 +/* don't suck in all of the win32 api */ +# define WIN32_LEAN_AND_MEAN 1 +#endif + +#ifndef LDAP_NEEDS_PROTOTYPES +/* force LDAP_P to always include prototypes */ +#define LDAP_NEEDS_PROTOTYPES 1 +#endif + +#ifndef LDAP_REL_ENG +#if (LDAP_VENDOR_VERSION == 000000) && !defined(LDAP_DEVEL) +#define LDAP_DEVEL +#endif +#if defined(LDAP_DEVEL) && !defined(LDAP_TEST) +#define LDAP_TEST +#endif +#endif + +#ifdef HAVE_STDDEF_H +# include +#endif + +#ifdef HAVE_EBCDIC +/* ASCII/EBCDIC converting replacements for stdio funcs + * vsnprintf and snprintf are used too, but they are already + * checked by the configure script + */ +#define fputs ber_pvt_fputs +#define fgets ber_pvt_fgets +#define printf ber_pvt_printf +#define fprintf ber_pvt_fprintf +#define vfprintf ber_pvt_vfprintf +#define vsprintf ber_pvt_vsprintf +#endif + +#include "ac/fdset.h" + +#include "ldap_cdefs.h" +#include "ldap_features.h" + +#include "ac/assert.h" +#include "ac/localize.h" + +#endif /* _LDAP_PORTABLE_H */ +/* end of portable.h.post */ + From 291c82e3fee97457e70addc316d9e1f8db764160 Mon Sep 17 00:00:00 2001 From: Nickolay Yastrebov Date: Fri, 2 Apr 2021 13:23:11 +0300 Subject: [PATCH 224/266] Fix translation for arrayFill, arrayReverseFill, arraySplit, arrayReverseSplit functions. #22377 --- .../functions/array-functions.md | 24 ++--- .../functions/array-functions.md | 88 +++++++++++++++++-- 2 files changed, 92 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 5e1d9d4ba23..499376a70d4 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -245,7 +245,7 @@ Elements set to `NULL` are handled as normal values. Returns the number of elements in the arr array for which func returns something other than 0. If ‘func’ is not specified, it returns the number of non-zero elements in the array. -Note that the `arrayCount` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. +Note that the `arrayCount` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. ## countEqual(arr, x) {#countequalarr-x} @@ -1229,7 +1229,7 @@ SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, └────────────────────────────────────┘ ``` -Note that the `arrayReverseFilter` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. +Note that the `arrayReverseFill` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. ## arraySplit(func, arr1, …) {#array-split} @@ -1293,7 +1293,7 @@ Note that the `arrayFirstIndex` is a [higher-order function](../../sql-reference ## arrayMin {#array-min} -Returns the minimum of elements in the source array. +Returns the minimum of elements in the source array. If the `func` function is specified, returns the mininum of elements converted by this function. @@ -1312,9 +1312,9 @@ arrayMin([func,] arr) **Returned value** -- The minimum of function values (or the array minimum). +- The minimum of function values (or the array minimum). -Type: if `func` is specified, matches `func` return value type, else matches the array elements type. +Type: if `func` is specified, matches `func` return value type, else matches the array elements type. **Examples** @@ -1348,7 +1348,7 @@ Result: ## arrayMax {#array-max} -Returns the maximum of elements in the source array. +Returns the maximum of elements in the source array. If the `func` function is specified, returns the maximum of elements converted by this function. @@ -1367,9 +1367,9 @@ arrayMax([func,] arr) **Returned value** -- The maximum of function values (or the array maximum). +- The maximum of function values (or the array maximum). -Type: if `func` is specified, matches `func` return value type, else matches the array elements type. +Type: if `func` is specified, matches `func` return value type, else matches the array elements type. **Examples** @@ -1403,7 +1403,7 @@ Result: ## arraySum {#array-sum} -Returns the sum of elements in the source array. +Returns the sum of elements in the source array. If the `func` function is specified, returns the sum of elements converted by this function. @@ -1418,7 +1418,7 @@ arraySum([func,] arr) **Arguments** - `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md). -- `arr` — Array. [Array](../../sql-reference/data-types/array.md). +- `arr` — Array. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -1458,7 +1458,7 @@ Result: ## arrayAvg {#array-avg} -Returns the average of elements in the source array. +Returns the average of elements in the source array. If the `func` function is specified, returns the average of elements converted by this function. @@ -1473,7 +1473,7 @@ arrayAvg([func,] arr) **Arguments** - `func` — Function. [Expression](../../sql-reference/data-types/special-data-types/expression.md). -- `arr` — Array. [Array](../../sql-reference/data-types/array.md). +- `arr` — Array. [Array](../../sql-reference/data-types/array.md). **Returned value** diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 4538941a4a4..560795506a0 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1111,6 +1111,78 @@ SELECT Функция `arrayFilter` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. +## arrayFill(func, arr1, …) {#array-fill} + +Перебирает `arr1` от первого элемента к последнему и заменяет `arr1[i]` на `arr1[i - 1]`, если `func` вернула 0. Первый элемент `arr1` остаётся неизменным. + +Примеры: + +``` sql +SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res +``` + +``` text +┌─res──────────────────────────────┐ +│ [1,1,3,11,12,12,12,5,6,14,14,14] │ +└──────────────────────────────────┘ +``` + +Функция `arrayFill` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. + +## arrayReverseFill(func, arr1, …) {#array-reverse-fill} + +Перебирает `arr1` от последнего элемента к первому и заменяет `arr1[i]` на `arr1[i + 1]`, если `func` вернула 0. Последний элемент `arr1` остаётся неизменным. + +Примеры: + +``` sql +SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, null, null]) AS res +``` + +``` text +┌─res────────────────────────────────┐ +│ [1,3,3,11,12,5,5,5,6,14,NULL,NULL] │ +└────────────────────────────────────┘ +``` + +Функция `arrayReverseFill` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. + +## arraySplit(func, arr1, …) {#array-split} + +Разделяет массив `arr1` на несколько. Если `func` возвращает не 0, то массив разделяется, а элемент помещается в левую часть. Массив не разбивается по первому элементу. + +Примеры: + +``` sql +SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res +``` + +``` text +┌─res─────────────┐ +│ [[1,2,3],[4,5]] │ +└─────────────────┘ +``` + +Функция `arraySplit` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. + +## arrayReverseSplit(func, arr1, …) {#array-reverse-split} + +Разделяет массив `arr1` на несколько. Если `func` возвращает не 0, то массив разделяется, а элемент помещается в правую часть. Массив не разбивается по последнему элементу. + +Примеры: + +``` sql +SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res +``` + +``` text +┌─res───────────────┐ +│ [[1],[2,3,4],[5]] │ +└───────────────────┘ +``` + +Функция `arrayReverseSplit` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. + ## arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1} Возвращает 1, если существует хотя бы один элемент массива `arr`, для которого функция func возвращает не 0. Иначе возвращает 0. @@ -1137,7 +1209,7 @@ SELECT ## arrayMin {#array-min} -Возвращает значение минимального элемента в исходном массиве. +Возвращает значение минимального элемента в исходном массиве. Если передана функция `func`, возвращается минимум из элементов массива, преобразованных этой функцией. @@ -1192,7 +1264,7 @@ SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res; ## arrayMax {#array-max} -Возвращает значение максимального элемента в исходном массиве. +Возвращает значение максимального элемента в исходном массиве. Если передана функция `func`, возвращается максимум из элементов массива, преобразованных этой функцией. @@ -1247,7 +1319,7 @@ SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res; ## arraySum {#array-sum} -Возвращает сумму элементов в исходном массиве. +Возвращает сумму элементов в исходном массиве. Если передана функция `func`, возвращается сумма элементов массива, преобразованных этой функцией. @@ -1262,7 +1334,7 @@ arraySum([func,] arr) **Аргументы** - `func` — функция. [Expression](../../sql-reference/data-types/special-data-types/expression.md). -- `arr` — массив. [Array](../../sql-reference/data-types/array.md). +- `arr` — массив. [Array](../../sql-reference/data-types/array.md). **Возвращаемое значение** @@ -1302,7 +1374,7 @@ SELECT arraySum(x -> x*x, [2, 3]) AS res; ## arrayAvg {#array-avg} -Возвращает среднее значение элементов в исходном массиве. +Возвращает среднее значение элементов в исходном массиве. Если передана функция `func`, возвращается среднее значение элементов массива, преобразованных этой функцией. @@ -1317,7 +1389,7 @@ arrayAvg([func,] arr) **Аргументы** - `func` — функция. [Expression](../../sql-reference/data-types/special-data-types/expression.md). -- `arr` — массив. [Array](../../sql-reference/data-types/array.md). +- `arr` — массив. [Array](../../sql-reference/data-types/array.md). **Возвращаемое значение** @@ -1355,7 +1427,7 @@ SELECT arrayAvg(x -> (x * x), [2, 4]) AS res; └─────┘ ``` -**Синтаксис** +**Синтаксис** ``` sql arraySum(arr) @@ -1367,7 +1439,7 @@ arraySum(arr) Тип: [Int](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). -**Аргументы** +**Аргументы** - `arr` — [массив](../../sql-reference/data-types/array.md). From 3492e3c40c9876932574859e9670dbd7f2199cea Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 2 Apr 2021 13:50:09 +0300 Subject: [PATCH 225/266] fixes --- ...ularity_versioned_collapsing_merge_tree.reference | 2 -- ...x_granularity_versioned_collapsing_merge_tree.sql | 12 ++++++++++-- tests/queries/skip_list.json | 1 + 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference index 067189f73fc..f93aae0225a 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference @@ -6,11 +6,9 @@ 4 1 0 -0 6 2 ----- 6 3 0 -0 diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql index 4d4dbda922d..44dd0412aea 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.sql @@ -62,7 +62,11 @@ OPTIMIZE TABLE four_rows_per_granule FINAL; SELECT COUNT(*) FROM four_rows_per_granule; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1; +-- We expect zero marks here, so we might get zero rows if all the parts were +-- deleted already. This can happen in parallel runs where there may be a long delay +-- between queries. So we must write the query in such a way that it always returns +-- zero rows if OK. +SELECT distinct(marks) d from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1 having d > 0; INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign, Version) VALUES ('2018-05-15', 1, 1000, 2000, 1, 1), ('2018-05-16', 2, 3000, 4000, 1, 1), ('2018-05-17', 3, 5000, 6000, 1, 1), ('2018-05-18', 4, 7000, 8000, 1, 1); @@ -120,6 +124,10 @@ OPTIMIZE TABLE six_rows_per_granule FINAL; SELECT COUNT(*) FROM six_rows_per_granule; -SELECT distinct(marks) from system.parts WHERE table = 'six_rows_per_granule' and database=currentDatabase() and active=1; +-- We expect zero marks here, so we might get zero rows if all the parts were +-- deleted already. This can happen in parallel runs where there may be a long delay +-- between queries. So we must write the query in such a way that it always returns +-- zero rows if OK. +SELECT distinct(marks) d from system.parts WHERE table = 'six_rows_per_granule' and database=currentDatabase() and active=1 having d > 0; DROP TABLE IF EXISTS six_rows_per_granule; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index df2090325a3..bdb5c5bd319 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -641,6 +641,7 @@ "01542_dictionary_load_exception_race", "01545_system_errors", // looks at the difference of values in system.errors "01560_optimize_on_insert_zookeeper", + "01563_distributed_query_finish", // looks at system.errors which is global "01575_disable_detach_table_of_dictionary", "01593_concurrent_alter_mutations_kill", "01593_concurrent_alter_mutations_kill_many_replicas", From 2952cb296de0145a458d08dcd55365b0165c0248 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Fri, 2 Apr 2021 13:57:46 +0300 Subject: [PATCH 226/266] Disable base64, fastops and libcpuid on unsupported platforms better --- cmake/find/base64.cmake | 4 ++-- cmake/find/fastops.cmake | 4 ++-- contrib/libcpuid-cmake/CMakeLists.txt | 10 ++++------ 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/cmake/find/base64.cmake b/cmake/find/base64.cmake index 1ac1d150a89..acade11eb2f 100644 --- a/cmake/find/base64.cmake +++ b/cmake/find/base64.cmake @@ -1,7 +1,7 @@ -if(NOT ARCH_PPC64LE) +if(ARCH_AMD64 OR ARCH_ARM) option (ENABLE_BASE64 "Enable base64" ${ENABLE_LIBRARIES}) elseif(ENABLE_BASE64) - message (${RECONFIGURE_MESSAGE_LEVEL} "base64 library is not supported on PowerPC") + message (${RECONFIGURE_MESSAGE_LEVEL} "base64 library is only supported on x86_64 and aarch64") endif() if (NOT ENABLE_BASE64) diff --git a/cmake/find/fastops.cmake b/cmake/find/fastops.cmake index 712ddc4a2c5..1675646654e 100644 --- a/cmake/find/fastops.cmake +++ b/cmake/find/fastops.cmake @@ -1,7 +1,7 @@ -if(NOT ARCH_ARM AND NOT OS_FREEBSD AND NOT OS_DARWIN AND NOT ARCH_PPC64LE) +if(ARCH_AMD64 AND NOT OS_FREEBSD AND NOT OS_DARWIN) option(ENABLE_FASTOPS "Enable fast vectorized mathematical functions library by Mikhail Parakhin" ${ENABLE_LIBRARIES}) elseif(ENABLE_FASTOPS) - message (${RECONFIGURE_MESSAGE_LEVEL} "Fastops library is not supported on ARM, FreeBSD and Darwin") + message (${RECONFIGURE_MESSAGE_LEVEL} "Fastops library is supported on x86_64 only, and not FreeBSD or Darwin") endif() if(NOT ENABLE_FASTOPS) diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index 952d0855a46..9baebb3ba1b 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -1,11 +1,9 @@ -if (NOT ARCH_ARM AND NOT ARCH_PPC64LE) +if(ARCH_AMD64) option (ENABLE_CPUID "Enable libcpuid library (only internal)" ${ENABLE_LIBRARIES}) -endif() - -if (ARCH_ARM AND ENABLE_CPUID) - message (${RECONFIGURE_MESSAGE_LEVEL} "cpuid is not supported on ARM") +elseif(ENABLE_CPUID) + message (${RECONFIGURE_MESSAGE_LEVEL} "libcpuid is only supported on x86_64") set (ENABLE_CPUID 0) -endif () +endif() if (NOT ENABLE_CPUID) add_library (cpuid INTERFACE) From 059175217f563c3387de2b9fdb132e3e8b3bb6b0 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 2 Apr 2021 13:57:56 +0300 Subject: [PATCH 227/266] Fix test a little bit --- tests/queries/0_stateless/01783_http_chunk_size.sh | 14 ++++++++------ tests/queries/skip_list.json | 3 ++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01783_http_chunk_size.sh b/tests/queries/0_stateless/01783_http_chunk_size.sh index 80b8bfa9e77..66ac4dfa975 100755 --- a/tests/queries/0_stateless/01783_http_chunk_size.sh +++ b/tests/queries/0_stateless/01783_http_chunk_size.sh @@ -4,12 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "DROP TABLE IF EXISTS table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- -echo "CREATE TABLE table (a String) ENGINE Memory()" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- +URL="${CLICKHOUSE_URL}&session_id=id_${CLICKHOUSE_DATABASE}" + +echo "DROP TABLE IF EXISTS table" | ${CLICKHOUSE_CURL} -sSg "${URL}" -d @- +echo "CREATE TABLE table (a String) ENGINE Memory()" | ${CLICKHOUSE_CURL} -sSg "${URL}" -d @- # NOTE: suppose that curl sends everything in a single chunk - there are no options to force the chunk-size. -echo "SET max_query_size=44" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- -echo -ne "INSERT INTO TABLE table FORMAT TabSeparated 1234567890 1234567890 1234567890 1234567890\n" | ${CLICKHOUSE_CURL} -H "Transfer-Encoding: chunked" -sS "${CLICKHOUSE_URL}&session_id=01783" --data-binary @- +echo "SET max_query_size=44" | ${CLICKHOUSE_CURL} -sSg "${URL}" -d @- +echo -ne "INSERT INTO TABLE table FORMAT TabSeparated 1234567890 1234567890 1234567890 1234567890\n" | ${CLICKHOUSE_CURL} -H "Transfer-Encoding: chunked" -sS "${URL}" --data-binary @- -echo "SELECT * from table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- -echo "DROP TABLE table" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&session_id=01783" -d @- +echo "SELECT * from table" | ${CLICKHOUSE_CURL} -sSg "${URL}" -d @- +echo "DROP TABLE table" | ${CLICKHOUSE_CURL} -sSg "${URL}" -d @- diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index df2090325a3..534b6ae5433 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -164,7 +164,8 @@ "00062_replicated_merge_tree_alter_zookeeper", /// Does not support renaming of multiple tables in single query "00634_rename_view", - "00140_rename" + "00140_rename", + "01783_http_chunk_size" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From 8a455d347a1587d4c35aa5e75cc7a9d86fb0c69d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 2 Apr 2021 14:08:32 +0300 Subject: [PATCH 228/266] Fix build --- src/Server/HTTP/HTTPServer.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Server/HTTP/HTTPServer.cpp b/src/Server/HTTP/HTTPServer.cpp index 3e050080bdd..5554a0ee31d 100644 --- a/src/Server/HTTP/HTTPServer.cpp +++ b/src/Server/HTTP/HTTPServer.cpp @@ -8,9 +8,9 @@ namespace DB HTTPServer::HTTPServer( const Context & context, HTTPRequestHandlerFactoryPtr factory_, - UInt16 portNumber, + UInt16 port_number, Poco::Net::HTTPServerParams::Ptr params) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), portNumber, params), factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), port_number, params), factory(factory_) { } @@ -26,10 +26,10 @@ HTTPServer::HTTPServer( HTTPServer::HTTPServer( const Context & context, HTTPRequestHandlerFactoryPtr factory_, - Poco::ThreadPool & threadPool, + Poco::ThreadPool & thread_pool, const Poco::Net::ServerSocket & socket, Poco::Net::HTTPServerParams::Ptr params) - : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), threadPool, socket, params), factory(factory_) + : TCPServer(new HTTPServerConnectionFactory(context, params, factory_), thread_pool, socket, params), factory(factory_) { } From 8011f5c36a4bbe97af71ae8d51a64f3befb1e49b Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Fri, 2 Apr 2021 14:19:25 +0300 Subject: [PATCH 229/266] DOCSUP-5910: Documented SimHash, MinHash, bitHammingDistance and tupleHammingDistance functions (#22131) Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> Co-authored-by: George Co-authored-by: Vladimir --- .../sql-reference/functions/bit-functions.md | 50 + .../sql-reference/functions/hash-functions.md | 937 ++++++++++++++++++ .../functions/tuple-functions.md | 53 +- .../sql-reference/functions/bit-functions.md | 50 + .../sql-reference/functions/hash-functions.md | 936 +++++++++++++++++ .../functions/tuple-functions.md | 52 + 6 files changed, 2077 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/bit-functions.md b/docs/en/sql-reference/functions/bit-functions.md index 31d09e48e01..e07f28c0f24 100644 --- a/docs/en/sql-reference/functions/bit-functions.md +++ b/docs/en/sql-reference/functions/bit-functions.md @@ -250,3 +250,53 @@ Result: └───────────────┘ ``` +## bitHammingDistance {#bithammingdistance} + +Returns the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) between the bit representations of two integer values. Can be used with [SimHash](../../sql-reference/functions/hash-functions.md#ngramsimhash) functions for detection of semi-duplicate strings. The smaller is the distance, the more likely those strings are the same. + +**Syntax** + +``` sql +bitHammingDistance(int1, int2) +``` + +**Arguments** + +- `int1` — First integer value. [Int64](../../sql-reference/data-types/int-uint.md). +- `int2` — Second integer value. [Int64](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- The Hamming distance. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Examples** + +Query: + +``` sql +SELECT bitHammingDistance(111, 121); +``` + +Result: + +``` text +┌─bitHammingDistance(111, 121)─┐ +│ 3 │ +└──────────────────────────────┘ +``` + +With [SimHash](../../sql-reference/functions/hash-functions.md#ngramsimhash): + +``` sql +SELECT bitHammingDistance(ngramSimHash('cat ate rat'), ngramSimHash('rat ate cat')); +``` + +Result: + +``` text +┌─bitHammingDistance(ngramSimHash('cat ate rat'), ngramSimHash('rat ate cat'))─┐ +│ 5 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 945ede4927f..c60067b06af 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -7,6 +7,8 @@ toc_title: Hash Hash functions can be used for the deterministic pseudo-random shuffling of elements. +Simhash is a hash function, which returns close hash values for close (similar) arguments. + ## halfMD5 {#hash-functions-halfmd5} [Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order. @@ -482,3 +484,938 @@ Result: - [xxHash](http://cyan4973.github.io/xxHash/). +## ngramSimHash {#ngramsimhash} + +Splits a ASCII string into n-grams of `ngramsize` symbols and returns the n-gram `simhash`. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +ngramSimHash(string[, ngramsize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT ngramSimHash('ClickHouse') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 1627567969 │ +└────────────┘ +``` + +## ngramSimHashCaseInsensitive {#ngramsimhashcaseinsensitive} + +Splits a ASCII string into n-grams of `ngramsize` symbols and returns the n-gram `simhash`. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +ngramSimHashCaseInsensitive(string[, ngramsize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT ngramSimHashCaseInsensitive('ClickHouse') AS Hash; +``` + +Result: + +``` text +┌──────Hash─┐ +│ 562180645 │ +└───────────┘ +``` + +## ngramSimHashUTF8 {#ngramsimhashutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and returns the n-gram `simhash`. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +ngramSimHashUTF8(string[, ngramsize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT ngramSimHashUTF8('ClickHouse') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 1628157797 │ +└────────────┘ +``` + +## ngramSimHashCaseInsensitiveUTF8 {#ngramsimhashcaseinsensitiveutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and returns the n-gram `simhash`. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +ngramSimHashCaseInsensitiveUTF8(string[, ngramsize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT ngramSimHashCaseInsensitiveUTF8('ClickHouse') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 1636742693 │ +└────────────┘ +``` + +## wordShingleSimHash {#wordshinglesimhash} + +Splits a ASCII string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +wordShingleSimHash(string[, shinglesize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT wordShingleSimHash('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 2328277067 │ +└────────────┘ +``` + +## wordShingleSimHashCaseInsensitive {#wordshinglesimhashcaseinsensitive} + +Splits a ASCII string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +wordShingleSimHashCaseInsensitive(string[, shinglesize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT wordShingleSimHashCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 2194812424 │ +└────────────┘ +``` + +## wordShingleSimHashUTF8 {#wordshinglesimhashutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +wordShingleSimHashUTF8(string[, shinglesize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optinal. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT wordShingleSimHashUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 2328277067 │ +└────────────┘ +``` + +## wordShingleSimHashCaseInsensitiveUTF8 {#wordshinglesimhashcaseinsensitiveutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + +**Syntax** + +``` sql +wordShingleSimHashCaseInsensitiveUTF8(string[, shinglesize]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT wordShingleSimHashCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Result: + +``` text +┌───────Hash─┐ +│ 2194812424 │ +└────────────┘ +``` + +## ngramMinHash {#ngramminhash} + +Splits a ASCII string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +ngramMinHash(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT ngramMinHash('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (18333312859352735453,9054248444481805918) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashCaseInsensitive {#ngramminhashcaseinsensitive} + +Splits a ASCII string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +ngramMinHashCaseInsensitive(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashCaseInsensitive('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (2106263556442004574,13203602793651726206) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashUTF8 {#ngramminhashutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +ngramMinHashUTF8(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashUTF8('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (18333312859352735453,6742163577938632877) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashCaseInsensitiveUTF8 {#ngramminhashcaseinsensitiveutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +ngramMinHashCaseInsensitiveUTF8(string [, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashCaseInsensitiveUTF8('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple───────────────────────────────────────┐ +│ (12493625717655877135,13203602793651726206) │ +└─────────────────────────────────────────────┘ +``` + +## ngramMinHashArg {#ngramminhasharg} + +Splits a ASCII string into n-grams of `ngramsize` symbols and returns the n-grams with minimum and maximum hashes, calculated by the [ngramMinHash](#ngramminhash) function with the same input. Is case sensitive. + +**Syntax** + +``` sql +ngramMinHashArg(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` n-grams each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashArg('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','Hou','kHo','use'),('Hou','lic','ick','ous','ckH','Cli')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgCaseInsensitive {#ngramminhashargcaseinsensitive} + +Splits a ASCII string into n-grams of `ngramsize` symbols and returns the n-grams with minimum and maximum hashes, calculated by the [ngramMinHashCaseInsensitive](#ngramminhashcaseinsensitive) function with the same input. Is case insensitive. + +**Syntax** + +``` sql +ngramMinHashArgCaseInsensitive(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` n-grams each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashArgCaseInsensitive('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','kHo','use','Cli'),('kHo','lic','ick','ous','ckH','Hou')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgUTF8 {#ngramminhashargutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and returns the n-grams with minimum and maximum hashes, calculated by the [ngramMinHashUTF8](#ngramminhashutf8) function with the same input. Is case sensitive. + +**Syntax** + +``` sql +ngramMinHashArgUTF8(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` n-grams each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashArgUTF8('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','Hou','kHo','use'),('kHo','Hou','lic','ick','ous','ckH')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgCaseInsensitiveUTF8 {#ngramminhashargcaseinsensitiveutf8} + +Splits a UTF-8 string into n-grams of `ngramsize` symbols and returns the n-grams with minimum and maximum hashes, calculated by the [ngramMinHashCaseInsensitiveUTF8](#ngramminhashcaseinsensitiveutf8) function with the same input. Is case insensitive. + +**Syntax** + +``` sql +ngramMinHashArgCaseInsensitiveUTF8(string[, ngramsize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — The size of an n-gram. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` n-grams each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT ngramMinHashArgCaseInsensitiveUTF8('ClickHouse') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ckH','ous','ick','lic','kHo','use'),('kHo','lic','ick','ous','ckH','Hou')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHash {#wordshingleminhash} + +Splits a ASCII string into parts (shingles) of `shinglesize` words and calculates hash values for each word shingle. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +wordShingleMinHash(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHash('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (16452112859864147620,5844417301642981317) │ +└────────────────────────────────────────────┘ +``` + +## wordShingleMinHashCaseInsensitive {#wordshingleminhashcaseinsensitive} + +Splits a ASCII string into parts (shingles) of `shinglesize` words and calculates hash values for each word shingle. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +wordShingleMinHashCaseInsensitive(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────┐ +│ (3065874883688416519,1634050779997673240) │ +└───────────────────────────────────────────┘ +``` + +## wordShingleMinHashUTF8 {#wordshingleminhashutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words and calculates hash values for each word shingle. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +wordShingleMinHashUTF8(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (16452112859864147620,5844417301642981317) │ +└────────────────────────────────────────────┘ +``` + +## wordShingleMinHashCaseInsensitiveUTF8 {#wordshingleminhashcaseinsensitiveutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words and calculates hash values for each word shingle. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case insensitive. + +Can be used for detection of semi-duplicate strings with [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). For two strings: if one of the returned hashes is the same for both strings, we think that those strings are the same. + +**Syntax** + +``` sql +wordShingleMinHashCaseInsensitiveUTF8(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two hashes — the minimum and the maximum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────┐ +│ (3065874883688416519,1634050779997673240) │ +└───────────────────────────────────────────┘ +``` + +## wordShingleMinHashArg {#wordshingleminhasharg} + +Splits a ASCII string into parts (shingles) of `shinglesize` words each and returns the shingles with minimum and maximum word hashes, calculated by the [wordshingleMinHash](#wordshingleminhash) function with the same input. Is case sensitive. + +**Syntax** + +``` sql +wordShingleMinHashArg(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` word shingles each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashArg('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────┐ +│ (('OLAP','database','analytical'),('online','oriented','processing')) │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgCaseInsensitive {#wordshingleminhashargcaseinsensitive} + +Splits a ASCII string into parts (shingles) of `shinglesize` words each and returns the shingles with minimum and maximum word hashes, calculated by the [wordShingleMinHashCaseInsensitive](#wordshingleminhashcaseinsensitive) function with the same input. Is case insensitive. + +**Syntax** + +``` sql +wordShingleMinHashArgCaseInsensitive(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` word shingles each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashArgCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────────────────────────────────┐ +│ (('queries','database','analytical'),('oriented','processing','DBMS')) │ +└────────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgUTF8 {#wordshingleminhashargutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words each and returns the shingles with minimum and maximum word hashes, calculated by the [wordShingleMinHashUTF8](#wordshingleminhashutf8) function with the same input. Is case sensitive. + +**Syntax** + +``` sql +wordShingleMinHashArgUTF8(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` word shingles each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashArgUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Result: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────┐ +│ (('OLAP','database','analytical'),('online','oriented','processing')) │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgCaseInsensitiveUTF8 {#wordshingleminhashargcaseinsensitiveutf8} + +Splits a UTF-8 string into parts (shingles) of `shinglesize` words each and returns the shingles with minimum and maximum word hashes, calculated by the [wordShingleMinHashCaseInsensitiveUTF8](#wordshingleminhashcaseinsensitiveutf8) function with the same input. Is case insensitive. + +**Syntax** + +``` sql +wordShingleMinHashArgCaseInsensitiveUTF8(string[, shinglesize, hashnum]) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — The number of minimum and maximum hashes used to calculate the result. Optional. Possible values: any number from `1` to `25`. Default value: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- Tuple with two tuples with `hashnum` word shingles each. + +Type: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Example** + +Query: + +``` sql +SELECT wordShingleMinHashArgCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Result: + +``` text +┌─Tuple──────────────────────────────────────────────────────────────────┐ +│ (('queries','database','analytical'),('oriented','processing','DBMS')) │ +└────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 884e1ef754f..86442835425 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -111,4 +111,55 @@ Result: - [Tuple](../../sql-reference/data-types/tuple.md) -[Original article](https://clickhouse.tech/docs/en/sql-reference/functions/tuple-functions/) +## tupleHammingDistance {#tuplehammingdistance} + +Returns the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) between two tuples of the same size. + +**Syntax** + +``` sql +tupleHammingDistance(tuple1, tuple2) +``` + +**Arguments** + +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). + +Tuples should have the same type of the elements. + +**Returned value** + +- The Hamming distance. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Examples** + +Query: + +``` sql +SELECT tupleHammingDistance((1, 2, 3), (3, 2, 1)) AS HammingDistance; +``` + +Result: + +``` text +┌─HammingDistance─┐ +│ 2 │ +└─────────────────┘ +``` + +Can be used with [MinHash](../../sql-reference/functions/hash-functions.md#ngramminhash) functions for detection of semi-duplicate strings: + +``` sql +SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseInsensitive(string)) as HammingDistance FROM (SELECT 'Clickhouse is a column-oriented database management system for online analytical processing of queries.' AS string); +``` + +Result: + +``` text +┌─HammingDistance─┐ +│ 2 │ +└─────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/bit-functions.md b/docs/ru/sql-reference/functions/bit-functions.md index 09844685a6c..a5124e67235 100644 --- a/docs/ru/sql-reference/functions/bit-functions.md +++ b/docs/ru/sql-reference/functions/bit-functions.md @@ -240,3 +240,53 @@ SELECT bitCount(333); └───────────────┘ ``` +## bitHammingDistance {#bithammingdistance} + +Возвращает [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между битовыми представлениями двух целых чисел. Может быть использовано с функциями [SimHash](../../sql-reference/functions/hash-functions.md#ngramsimhash) для проверки двух строк на схожесть. Чем меньше расстояние, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +bitHammingDistance(int1, int2) +``` + +**Аргументы** + +- `int1` — первое целое число. [Int64](../../sql-reference/data-types/int-uint.md). +- `int2` — второе целое число. [Int64](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Расстояние Хэмминга. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +SELECT bitHammingDistance(111, 121); +``` + +Результат: + +``` text +┌─bitHammingDistance(111, 121)─┐ +│ 3 │ +└──────────────────────────────┘ +``` + +Используя [SimHash](../../sql-reference/functions/hash-functions.md#ngramsimhash): + +``` sql +SELECT bitHammingDistance(ngramSimHash('cat ate rat'), ngramSimHash('rat ate cat')); +``` + +Результат: + +``` text +┌─bitHammingDistance(ngramSimHash('cat ate rat'), ngramSimHash('rat ate cat'))─┐ +│ 5 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 6797f530346..2efff9c3727 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -7,6 +7,8 @@ toc_title: "Функции хэширования" Функции хэширования могут использоваться для детерминированного псевдослучайного разбрасывания элементов. +Simhash – это хеш-функция, которая для близких значений возвращает близкий хеш. + ## halfMD5 {#hash-functions-halfmd5} [Интерпретирует](../../sql-reference/functions/hash-functions.md#type_conversion_functions-reinterpretAsString) все входные параметры как строки и вычисляет хэш [MD5](https://ru.wikipedia.org/wiki/MD5) для каждой из них. Затем объединяет хэши, берет первые 8 байт хэша результирующей строки и интерпретирует их как значение типа `UInt64` с big-endian порядком байтов. @@ -484,3 +486,937 @@ SELECT xxHash32('Hello, world!'); - [xxHash](http://cyan4973.github.io/xxHash/). +## ngramSimHash {#ngramsimhash} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммовый `simhash`. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +ngramSimHash(string[, ngramsize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT ngramSimHash('ClickHouse') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 1627567969 │ +└────────────┘ +``` + +## ngramSimHashCaseInsensitive {#ngramsimhashcaseinsensitive} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммовый `simhash`. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +ngramSimHashCaseInsensitive(string[, ngramsize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT ngramSimHashCaseInsensitive('ClickHouse') AS Hash; +``` + +Результат: + +``` text +┌──────Hash─┐ +│ 562180645 │ +└───────────┘ +``` + +## ngramSimHashUTF8 {#ngramsimhashutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммовый `simhash`. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +ngramSimHashUTF8(string[, ngramsize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT ngramSimHashUTF8('ClickHouse') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 1628157797 │ +└────────────┘ +``` + +## ngramSimHashCaseInsensitiveUTF8 {#ngramsimhashcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммовый `simhash`. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +ngramSimHashCaseInsensitiveUTF8(string[, ngramsize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT ngramSimHashCaseInsensitiveUTF8('ClickHouse') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 1636742693 │ +└────────────┘ +``` + +## wordShingleSimHash {#wordshinglesimhash} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и возвращает шингловый `simhash`. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +wordShingleSimHash(string[, shinglesize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleSimHash('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 2328277067 │ +└────────────┘ +``` + +## wordShingleSimHashCaseInsensitive {#wordshinglesimhashcaseinsensitive} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и возвращает шингловый `simhash`. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +wordShingleSimHashCaseInsensitive(string[, shinglesize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleSimHashCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 2194812424 │ +└────────────┘ +``` + +## wordShingleSimHashUTF8 {#wordshinglesimhashutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и возвращает шингловый `simhash`. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +wordShingleSimHashUTF8(string[, shinglesize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleSimHashUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 2328277067 │ +└────────────┘ +``` + +## wordShingleSimHashCaseInsensitiveUTF8 {#wordshinglesimhashcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и возвращает шингловый `simhash`. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [bitHammingDistance](../../sql-reference/functions/bit-functions.md#bithammingdistance). Чем меньше [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между результатом вычисления `simhash` двух строк, тем больше вероятность, что строки совпадают. + +**Синтаксис** + +``` sql +wordShingleSimHashCaseInsensitiveUTF8(string[, shinglesize]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Значение хеш-функции от строки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleSimHashCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Hash; +``` + +Результат: + +``` text +┌───────Hash─┐ +│ 2194812424 │ +└────────────┘ +``` + +## ngramMinHash {#ngramminhash} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и вычисляет хеш для каждой n-граммы. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +ngramMinHash(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHash('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (18333312859352735453,9054248444481805918) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashCaseInsensitive {#ngramminhashcaseinsensitive} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и вычисляет хеш для каждой n-граммы. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +ngramMinHashCaseInsensitive(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashCaseInsensitive('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (2106263556442004574,13203602793651726206) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashUTF8 {#ngramminhashutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и вычисляет хеш для каждой n-граммы. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** +``` sql +ngramMinHashUTF8(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashUTF8('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (18333312859352735453,6742163577938632877) │ +└────────────────────────────────────────────┘ +``` + +## ngramMinHashCaseInsensitiveUTF8 {#ngramminhashcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и вычисляет хеш для каждой n-граммы. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +ngramMinHashCaseInsensitiveUTF8(string [, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashCaseInsensitiveUTF8('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple───────────────────────────────────────┐ +│ (12493625717655877135,13203602793651726206) │ +└─────────────────────────────────────────────┘ +``` + +## ngramMinHashArg {#ngramminhasharg} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммы с минимальным и максимальным хешами, вычисленными функцией [ngramMinHash](#ngramminhash) с теми же входными данными. Функция регистрозависимая. + +**Синтаксис** + +``` sql +ngramMinHashArg(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` n-грамм. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashArg('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','Hou','kHo','use'),('Hou','lic','ick','ous','ckH','Cli')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgCaseInsensitive {#ngramminhashargcaseinsensitive} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммы с минимальным и максимальным хешами, вычисленными функцией [ngramMinHashCaseInsensitive](#ngramminhashcaseinsensitive) с теми же входными данными. Функция регистро**не**зависимая. + +**Синтаксис** + +``` sql +ngramMinHashArgCaseInsensitive(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` n-грамм. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashArgCaseInsensitive('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','kHo','use','Cli'),('kHo','lic','ick','ous','ckH','Hou')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgUTF8 {#ngramminhashargutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммы с минимальным и максимальным хешами, вычисленными функцией [ngramMinHashUTF8](#ngramminhashutf8) с теми же входными данными. Функция регистрозависимая. + +**Синтаксис** + +``` sql +ngramMinHashArgUTF8(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` n-грамм. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashArgUTF8('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ous','ick','lic','Hou','kHo','use'),('kHo','Hou','lic','ick','ous','ckH')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## ngramMinHashArgCaseInsensitiveUTF8 {#ngramminhashargcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (n-граммы) размером `ngramsize` символов и возвращает n-граммы с минимальным и максимальным хешами, вычисленными функцией [ngramMinHashCaseInsensitiveUTF8](#ngramminhashcaseinsensitiveutf8) с теми же входными данными. Функция регистро**не**зависимая. + +**Синтаксис** + +``` sql +ngramMinHashArgCaseInsensitiveUTF8(string[, ngramsize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `ngramsize` — размер n-грамм. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` n-грамм. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT ngramMinHashArgCaseInsensitiveUTF8('ClickHouse') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────────────┐ +│ (('ckH','ous','ick','lic','kHo','use'),('kHo','lic','ick','ous','ckH','Hou')) │ +└───────────────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHash {#wordshingleminhash} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и вычисляет хеш для каждого шингла. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +wordShingleMinHash(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHash('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (16452112859864147620,5844417301642981317) │ +└────────────────────────────────────────────┘ +``` + +## wordShingleMinHashCaseInsensitive {#wordshingleminhashcaseinsensitive} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и вычисляет хеш для каждого шингла. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +wordShingleMinHashCaseInsensitive(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────┐ +│ (3065874883688416519,1634050779997673240) │ +└───────────────────────────────────────────┘ +``` + +## wordShingleMinHashUTF8 {#wordshingleminhashutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и вычисляет хеш для каждого шингла. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистрозависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +wordShingleMinHashUTF8(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────┐ +│ (16452112859864147620,5844417301642981317) │ +└────────────────────────────────────────────┘ +``` + +## wordShingleMinHashCaseInsensitiveUTF8 {#wordshingleminhashcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и вычисляет хеш для каждого шингла. Использует `hashnum` минимальных хешей, чтобы вычислить минимальный хеш, и `hashnum` максимальных хешей, чтобы вычислить максимальный хеш. Возвращает кортеж из этих хешей. Функция регистро**не**зависимая. + +Может быть использована для проверки двух строк на схожесть вместе с функцией [tupleHammingDistance](../../sql-reference/functions/tuple-functions.md#tuplehammingdistance). Если для двух строк минимальные или максимальные хеши одинаковы, мы считаем, что эти строки совпадают. + +**Синтаксис** + +``` sql +wordShingleMinHashCaseInsensitiveUTF8(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж с двумя хешами — минимальным и максимальным. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([UInt64](../../sql-reference/data-types/int-uint.md), [UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).') AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────┐ +│ (3065874883688416519,1634050779997673240) │ +└───────────────────────────────────────────┘ +``` + +## wordShingleMinHashArg {#wordshingleminhasharg} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и возвращает шинглы с минимальным и максимальным хешами, вычисленными функцией [wordshingleMinHash](#wordshingleminhash) с теми же входными данными. Функция регистрозависимая. + +**Синтаксис** + +``` sql +wordShingleMinHashArg(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` шинглов. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashArg('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────┐ +│ (('OLAP','database','analytical'),('online','oriented','processing')) │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgCaseInsensitive {#wordshingleminhashargcaseinsensitive} + +Выделяет из ASCII строки отрезки (шинглы) из `shinglesize` слов и возвращает шинглы с минимальным и максимальным хешами, вычисленными функцией [wordShingleMinHashCaseInsensitive](#wordshingleminhashcaseinsensitive) с теми же входными данными. Функция регистро**не**зависимая. + +**Синтаксис** + +``` sql +wordShingleMinHashArgCaseInsensitive(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` шинглов. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashArgCaseInsensitive('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────────────────────────────────┐ +│ (('queries','database','analytical'),('oriented','processing','DBMS')) │ +└────────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgUTF8 {#wordshingleminhashargutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и возвращает шинглы с минимальным и максимальным хешами, вычисленными функцией [wordShingleMinHashUTF8](#wordshingleminhashutf8) с теми же входными данными. Функция регистрозависимая. + +**Синтаксис** + +``` sql +wordShingleMinHashArgUTF8(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` шинглов. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashArgUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Результат: + +``` text +┌─Tuple─────────────────────────────────────────────────────────────────┐ +│ (('OLAP','database','analytical'),('online','oriented','processing')) │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +## wordShingleMinHashArgCaseInsensitiveUTF8 {#wordshingleminhashargcaseinsensitiveutf8} + +Выделяет из UTF-8 строки отрезки (шинглы) из `shinglesize` слов и возвращает шинглы с минимальным и максимальным хешами, вычисленными функцией [wordShingleMinHashCaseInsensitiveUTF8](#wordshingleminhashcaseinsensitiveutf8) с теми же входными данными. Функция регистро**не**зависимая. + +**Синтаксис** + +``` sql +wordShingleMinHashArgCaseInsensitiveUTF8(string[, shinglesize, hashnum]) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md). +- `shinglesize` — размер словесных шинглов. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `3`. [UInt8](../../sql-reference/data-types/int-uint.md). +- `hashnum` — количество минимальных и максимальных хешей, которое используется при вычислении результата. Необязательный. Возможные значения: любое число от `1` до `25`. Значение по умолчанию: `6`. [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Кортеж из двух кортежей, каждый из которых состоит из `hashnum` шинглов. + +Тип: [Tuple](../../sql-reference/data-types/tuple.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md)), [Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md))). + +**Пример** + +Запрос: + +``` sql +SELECT wordShingleMinHashArgCaseInsensitiveUTF8('ClickHouse® is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).', 1, 3) AS Tuple; +``` + +Результат: + +``` text +┌─Tuple──────────────────────────────────────────────────────────────────┐ +│ (('queries','database','analytical'),('oriented','processing','DBMS')) │ +└────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index a56eac27db2..381743a450b 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -111,3 +111,55 @@ SELECT untuple((* EXCEPT (v2, v3),)) FROM kv; - [Tuple](../../sql-reference/data-types/tuple.md) +## tupleHammingDistance {#tuplehammingdistance} + +Возвращает [расстояние Хэмминга](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%A5%D1%8D%D0%BC%D0%BC%D0%B8%D0%BD%D0%B3%D0%B0) между двумя кортежами одинакового размера. + +**Синтаксис** + +``` sql +tupleHammingDistance(tuple1, tuple2) +``` + +**Аргументы** + +- `tuple1` — первый кортеж. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — второй кортеж. [Tuple](../../sql-reference/data-types/tuple.md). + +Кортежи должны иметь одинаковый размер и тип элементов. + +**Возвращаемое значение** + +- Расстояние Хэмминга. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +SELECT tupleHammingDistance((1, 2, 3), (3, 2, 1)) AS HammingDistance; +``` + +Результат: + +``` text +┌─HammingDistance─┐ +│ 2 │ +└─────────────────┘ +``` + +Может быть использовано с функциями [MinHash](../../sql-reference/functions/hash-functions.md#ngramminhash) для проверки строк на совпадение: + +``` sql +SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseInsensitive(string)) as HammingDistance FROM (SELECT 'Clickhouse is a column-oriented database management system for online analytical processing of queries.' AS string); +``` + +Результат: + +``` text +┌─HammingDistance─┐ +│ 2 │ +└─────────────────┘ +``` From c1ab9de72084d0d65af24e2d4d8d521426915656 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 2 Apr 2021 15:57:30 +0300 Subject: [PATCH 230/266] Update MergeTreeDataPartChecksum.cpp --- src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 29ff4626a17..b0eb1cbea70 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -293,7 +293,7 @@ String MergeTreeDataPartChecksums::getTotalChecksumHex() const { SipHash hash_of_all_files; - for (auto && [name, checksum] : files) + for (const auto & [name, checksum] : files) { updateHash(hash_of_all_files, name); hash_of_all_files.update(checksum.file_hash); From 9e342bf9f0c698a27a0d6f628181eb97f2c90a75 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Fri, 2 Apr 2021 17:31:20 +0400 Subject: [PATCH 231/266] Revisit macOS build instructions --- docs/en/development/build-osx.md | 64 ++++++++++++++++++++++++-------- 1 file changed, 49 insertions(+), 15 deletions(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index e0b1be710f1..956c3d05ee3 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -5,43 +5,77 @@ toc_title: Build on Mac OS X # How to Build ClickHouse on Mac OS X {#how-to-build-clickhouse-on-mac-os-x} -Build should work on Mac OS X 10.15 (Catalina). +Build should work on macOS 10.15 (Catalina) and higher based on x86_64 (Intel based Macs) architecture with recent Xcode's native AppleClang, or Homebrew's vanilla Clang or GCC compilers. ## Install Homebrew {#install-homebrew} ``` bash -$ /usr/bin/ruby -e "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/master/install)" +$ /bin/bash -c "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/HEAD/install.sh)" ``` +## Install Xcode and Command Line Tools {#install-xcode-and-command-line-tools} + +Install the lates [Xcode](https://apps.apple.com/am/app/xcode/id497799835?mt=12) from App Store. + +Open it at least once to accept the end-user license agreement and automatically install the required components. + +Then, make sure that the latest Comman Line Tools are installed and selected in the system: + +``` bash +$ sudo rm -rf /Library/Developer/CommandLineTools +$ sudo xcode-select --install +``` + +Reboot. + ## Install Required Compilers, Tools, and Libraries {#install-required-compilers-tools-and-libraries} ``` bash -$ brew install cmake ninja libtool gettext llvm +$ brew update +$ brew install cmake ninja libtool gettext llvm gcc ``` ## Checkout ClickHouse Sources {#checkout-clickhouse-sources} ``` bash -$ git clone --recursive git@github.com:ClickHouse/ClickHouse.git -``` - -or - -``` bash -$ git clone --recursive https://github.com/ClickHouse/ClickHouse.git - -$ cd ClickHouse +$ git clone --recursive git@github.com:ClickHouse/ClickHouse.git # or https://github.com/ClickHouse/ClickHouse.git ``` ## Build ClickHouse {#build-clickhouse} -> Please note: ClickHouse doesn't support build with native Apple Clang compiler, we need use clang from LLVM. +To build using Xcode's native AppleClang compiler: ``` bash +$ cd ClickHouse +$ rm -rf build $ mkdir build $ cd build -$ cmake .. -DCMAKE_C_COMPILER=`brew --prefix llvm`/bin/clang -DCMAKE_CXX_COMPILER=`brew --prefix llvm`/bin/clang++ -DCMAKE_PREFIX_PATH=`brew --prefix llvm` -$ ninja +$ cmake -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. +$ cmake --build . --config RelWithDebInfo +$ cd .. +``` + +To build using Homebrew's vanilla Clang compiler: + +``` bash +$ cd ClickHouse +$ rm -rf build +$ mkdir build +$ cd build +$ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER==$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. +$ cmake --build . --config RelWithDebInfo +$ cd .. +``` + +To build using Homebrew's vanilla GCC compiler: + +``` bash +$ cd ClickHouse +$ rm -rf build +$ mkdir build +$ cd build +$ cmake -DCMAKE_C_COMPILER=$(brew --prefix gcc)/bin/gcc-10 -DCMAKE_CXX_COMPILER=$(brew --prefix gcc)/bin/g++-10 -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. +$ cmake --build . --config RelWithDebInfo $ cd .. ``` From ea45779f0c7486718e57c6f574e046a8c40fb1c4 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Fri, 2 Apr 2021 17:38:17 +0400 Subject: [PATCH 232/266] Adjust wording --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 956c3d05ee3..e220ca612c8 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -5,7 +5,7 @@ toc_title: Build on Mac OS X # How to Build ClickHouse on Mac OS X {#how-to-build-clickhouse-on-mac-os-x} -Build should work on macOS 10.15 (Catalina) and higher based on x86_64 (Intel based Macs) architecture with recent Xcode's native AppleClang, or Homebrew's vanilla Clang or GCC compilers. +Build should work on x86_64 (Intel) based macOS 10.15 (Catalina) and higher with recent Xcode's native AppleClang, or Homebrew's vanilla Clang or GCC compilers. ## Install Homebrew {#install-homebrew} From 8cd841c4acd6ea8aeee33ab21ac0c9c2c1245f0e Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Apr 2021 16:55:42 +0300 Subject: [PATCH 233/266] Handle lowcardinality block in MergeJoin extractMinMax --- src/Interpreters/MergeJoin.cpp | 9 +++++---- src/Interpreters/MergeJoin.h | 3 +++ 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index ddeaf053225..e9d1eafbdb9 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -86,10 +86,10 @@ Block extractMinMax(const Block & block, const Block & keys) for (size_t i = 0; i < columns.size(); ++i) { - const auto & src_column = block.getByName(keys.getByPosition(i).name); - - columns[i]->insertFrom(*src_column.column, 0); - columns[i]->insertFrom(*src_column.column, block.rows() - 1); + const auto & src_column = block.getByName(min_max.getByPosition(i).name); + /// Cannot use insertFrom because src_column type can differ from keys, e.g. to be LowCardinality + columns[i]->insert((*src_column.column)[0]); + columns[i]->insert((*src_column.column)[block.rows() - 1]); } min_max.setColumns(std::move(columns)); @@ -485,6 +485,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); } +/// Has to be called event totals is empty void MergeJoin::setTotals(const Block & totals_block) { totals = totals_block; diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index a13d0304907..d145a69ce9d 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -76,12 +76,15 @@ private: Block right_table_keys; Block right_columns_to_add; SortedBlocksWriter::Blocks right_blocks; + + /// Each block stores first and last row from corresponding sorted block on disk Blocks min_max_right_blocks; std::shared_ptr left_blocks_buffer; std::shared_ptr used_rows_bitmap; mutable std::unique_ptr cached_right_blocks; std::vector> loaded_right_blocks; std::unique_ptr disk_writer; + /// Set of files with sorted blocks SortedBlocksWriter::SortedFiles flushed_right_blocks; Block totals; std::atomic is_in_memory{true}; From c8f51998bacfea97a58d489edde11cdd1b456ac5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Apr 2021 17:51:02 +0300 Subject: [PATCH 234/266] Remove LowCardinality from right block sample in MergeJoin --- src/Interpreters/MergeJoin.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index e9d1eafbdb9..02c683b28d0 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -76,6 +76,7 @@ int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, return left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); } +/// Get first and last row from sorted block Block extractMinMax(const Block & block, const Block & keys) { if (block.rows() == 0) @@ -87,9 +88,9 @@ Block extractMinMax(const Block & block, const Block & keys) for (size_t i = 0; i < columns.size(); ++i) { const auto & src_column = block.getByName(min_max.getByPosition(i).name); - /// Cannot use insertFrom because src_column type can differ from keys, e.g. to be LowCardinality - columns[i]->insert((*src_column.column)[0]); - columns[i]->insert((*src_column.column)[block.rows() - 1]); + + columns[i]->insertFrom(*src_column.column, 0); + columns[i]->insertFrom(*src_column.column, block.rows() - 1); } min_max.setColumns(std::move(columns)); @@ -465,6 +466,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right table_join->splitAdditionalColumns(right_sample_block, right_table_keys, right_columns_to_add); JoinCommon::removeLowCardinalityInplace(right_table_keys); + JoinCommon::removeLowCardinalityInplace(right_sample_block, table_join->keyNamesRight()); const NameSet required_right_keys = table_join->requiredRightKeys(); for (const auto & column : right_table_keys) From 414d6fb25d59aa1c71d7818a6c3a3acc8e4100f7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 2 Apr 2021 18:08:20 +0300 Subject: [PATCH 235/266] Update skip_list.json --- tests/queries/skip_list.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index bdb5c5bd319..37ed408287a 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -668,6 +668,7 @@ "01702_system_query_log", // Runs many global system queries "01715_background_checker_blather_zookeeper", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. + "01722_long_brotli_http_compression_json_format", // it is broken in some unimaginable way with the genius error "cannot write to ofstream", not sure how to debug this "01747_alter_partition_key_enum_zookeeper", "01748_dictionary_table_dot", // creates database "01760_polygon_dictionaries", From 8dcfe119bf1197751c41ff06fc34be169189d95c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 2 Apr 2021 18:18:36 +0300 Subject: [PATCH 236/266] Add test pmj_lc_bug --- .../queries/0_stateless/01785_pmj_lc_bug.reference | 4 ++++ tests/queries/0_stateless/01785_pmj_lc_bug.sql | 14 ++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/01785_pmj_lc_bug.reference create mode 100644 tests/queries/0_stateless/01785_pmj_lc_bug.sql diff --git a/tests/queries/0_stateless/01785_pmj_lc_bug.reference b/tests/queries/0_stateless/01785_pmj_lc_bug.reference new file mode 100644 index 00000000000..98fb6a68656 --- /dev/null +++ b/tests/queries/0_stateless/01785_pmj_lc_bug.reference @@ -0,0 +1,4 @@ +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01785_pmj_lc_bug.sql b/tests/queries/0_stateless/01785_pmj_lc_bug.sql new file mode 100644 index 00000000000..722faa9b40d --- /dev/null +++ b/tests/queries/0_stateless/01785_pmj_lc_bug.sql @@ -0,0 +1,14 @@ +SET join_algorithm = 'partial_merge'; +SET max_bytes_in_join = '100'; + +CREATE TABLE foo_lc (n LowCardinality(String)) ENGINE = Memory; +CREATE TABLE foo (n String) ENGINE = Memory; + +INSERT INTO foo SELECT toString(number) AS n FROM system.numbers LIMIT 1025; +INSERT INTO foo_lc SELECT toString(number) AS n FROM system.numbers LIMIT 1025; + +SELECT 1025 == count(n) FROM foo_lc AS t1 ANY LEFT JOIN foo_lc AS t2 ON t1.n == t2.n; +SELECT 1025 == count(n) FROM foo AS t1 ANY LEFT JOIN foo_lc AS t2 ON t1.n == t2.n; +SELECT 1025 == count(n) FROM foo_lc AS t1 ANY LEFT JOIN foo AS t2 ON t1.n == t2.n; + +SELECT 1025 == count(n) FROM foo_lc AS t1 ALL LEFT JOIN foo_lc AS t2 ON t1.n == t2.n; From f6d6f568c2f90f5ca9040dd45fd44c2b918360c9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Apr 2021 18:32:01 +0300 Subject: [PATCH 237/266] Auto version update to [21.4.1.6422] [54449] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index bd7885bc41b..2b13983e600 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -3,7 +3,7 @@ SET(VERSION_REVISION 54449) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 4) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH af2135ef9dc72f16fa4f229b731262c3f0a8bbdc) -SET(VERSION_DESCRIBE v21.4.1.1-prestable) -SET(VERSION_STRING 21.4.1.1) +SET(VERSION_GITHASH 3827789b3d8fd2021952e57e5110343d26daa1a1) +SET(VERSION_DESCRIBE v21.4.1.6422-prestable) +SET(VERSION_STRING 21.4.1.6422) # end of autochange From 39b5674bef8dc8fb88e971a85a4cf6f1e9224ea2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Apr 2021 18:34:29 +0300 Subject: [PATCH 238/266] Auto version update to [21.5.1.1] [54450] --- cmake/autogenerated_versions.txt | 8 +++--- debian/changelog | 4 +-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../StorageSystemContributors.generated.cpp | 26 +++++++++++++++++++ 6 files changed, 35 insertions(+), 9 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 2b13983e600..9d74179902d 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54449) +SET(VERSION_REVISION 54450) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 4) +SET(VERSION_MINOR 5) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 3827789b3d8fd2021952e57e5110343d26daa1a1) -SET(VERSION_DESCRIBE v21.4.1.6422-prestable) -SET(VERSION_STRING 21.4.1.6422) +SET(VERSION_DESCRIBE v21.5.1.1-prestable) +SET(VERSION_STRING 21.5.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index 23d63b41099..be77dfdefe9 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.4.1.1) unstable; urgency=low +clickhouse (21.5.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Sat, 06 Mar 2021 14:43:27 +0300 + -- clickhouse-release Fri, 02 Apr 2021 18:34:26 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index d9cd68254b7..2efba9735ae 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.4.1.* +ARG version=21.5.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index d22516eab0a..05ca29f22d4 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.4.1.* +ARG version=21.5.1.* ARG gosu_ver=1.10 # set non-empty deb_location_url url to create a docker image diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index e727d2a3ecf..976c46ebe27 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.4.1.* +ARG version=21.5.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index fd4807e550c..46ead225102 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -4,6 +4,7 @@ const char * auto_contributors[] { "20018712", "243f6a88 85a308d3", "243f6a8885a308d313198a2e037", + "3ldar-nasyrov", "821008736@qq.com", "Akazz", "Alain BERRIER", @@ -58,6 +59,7 @@ const char * auto_contributors[] { "Alexey Vasiliev", "Alexey Zatelepin", "Alexsey Shestakov", + "Ali Demirci", "Aliaksandr Pliutau", "Aliaksandr Shylau", "Amos Bird", @@ -138,6 +140,7 @@ const char * auto_contributors[] { "Brett Hoerner", "Bulat Gaifullin", "Carbyn", + "Chao Ma", "Chao Wang", "Chen Yufei", "Chienlung Cheung", @@ -258,6 +261,7 @@ const char * auto_contributors[] { "Ilya Skrypitsa", "Ilya Yatsishin", "ImgBotApp", + "Islam Israfilov", "Islam Israfilov (Islam93)", "Ivan", "Ivan A. Torgashov", @@ -367,6 +371,7 @@ const char * auto_contributors[] { "Mikahil Nacharov", "Mike", "Mike F", + "Mike Kot", "Mikhail", "Mikhail Cheshkov", "Mikhail Fandyushin", @@ -378,6 +383,7 @@ const char * auto_contributors[] { "Mikhail Salosin", "Mikhail Surin", "Mikhail f. Shiryaev", + "MikuSugar", "Milad Arabi", "Mohammad Hossein Sekhavat", "MovElb", @@ -388,6 +394,7 @@ const char * auto_contributors[] { "Narek Galstyan", "NeZeD [Mac Pro]", "Neeke Gao", + "Neng Liu", "Nico Mandery", "Nico Piderman", "Nicolae Vartolomei", @@ -439,6 +446,7 @@ const char * auto_contributors[] { "Philippe Ombredanne", "Potya", "Pradeep Chhetri", + "Pysaoke", "Quid37", "Rafael David Tinoco", "Ramazan Polat", @@ -455,6 +463,7 @@ const char * auto_contributors[] { "Roman Peshkurov", "Roman Tsisyk", "Ruslan", + "Ruslan Savchenko", "Russ Frank", "Ruzal Ibragimov", "S.M.A. Djawadi", @@ -468,6 +477,7 @@ const char * auto_contributors[] { "Sergei Semin", "Sergei Shtykov", "Sergei Tsetlin (rekub)", + "Sergey Demurin", "Sergey Elantsev", "Sergey Fedorov", "Sergey Kononenko", @@ -483,6 +493,7 @@ const char * auto_contributors[] { "SevaCode", "Sherry Wang", "Silviu Caragea", + "Simeon Emanuilov", "Simon Liu", "Simon Podlipsky", "Sina", @@ -504,7 +515,9 @@ const char * auto_contributors[] { "TCeason", "Tagir Kuskarov", "Tai White", + "Taleh Zaliyev", "Tangaev", + "Tatiana Kirillova", "Tema Novikov", "The-Alchemist", "TiunovNN", @@ -534,6 +547,7 @@ const char * auto_contributors[] { "Veselkov Konstantin", "Victor Tarnavsky", "Viktor Taranenko", + "Vitaliy Fedorchenko", "Vitaliy Karnienko", "Vitaliy Kozlovskiy", "Vitaliy Lyudvichenko", @@ -566,6 +580,7 @@ const char * auto_contributors[] { "William Shallum", "Winter Zhang", "Xianda Ke", + "Xiang Zhou", "Y Lu", "Yangkuan Liu", "Yatsishin Ilya", @@ -683,6 +698,7 @@ const char * auto_contributors[] { "frank", "franklee", "fredchenbj", + "fuqi", "fuwhu", "g-arslan", "ggerogery", @@ -701,8 +717,10 @@ const char * auto_contributors[] { "idfer", "igor", "igor.lapko", + "ikarishinjieva", "ikopylov", "imgbot[bot]", + "ip", "it1804", "ivan-kush", "ivanzhukov", @@ -715,6 +733,8 @@ const char * auto_contributors[] { "jianmei zhang", "jyz0309", "keenwolf", + "kevin wan", + "kirillikoff", "kmeaw", "koshachy", "kreuzerkrieg", @@ -744,6 +764,7 @@ const char * auto_contributors[] { "malkfilipp", "manmitya", "maqroll", + "mastertheknife", "maxim", "maxim-babenko", "maxkuzn", @@ -754,6 +775,7 @@ const char * auto_contributors[] { "mergify[bot]", "mf5137", "mfridental", + "michon470", "miha-g", "mikepop7", "millb", @@ -791,6 +813,7 @@ const char * auto_contributors[] { "r1j1k", "rainbowsysu", "ritaank", + "robert", "robot-clickhouse", "robot-metrika-test", "rodrigargar", @@ -850,6 +873,7 @@ const char * auto_contributors[] { "ygrek", "yhgcn", "yiguolei", + "yingjinghan", "ylchou", "yonesko", "yuefoo", @@ -863,6 +887,7 @@ const char * auto_contributors[] { "zhen ni", "zhukai", "zlx19950903", + "zvonand", "zvrr", "zvvr", "zzsmdfj", @@ -879,6 +904,7 @@ const char * auto_contributors[] { "张健", "张风啸", "徐炘", + "曲正鹏", "极客青年", "谢磊", "贾顺名(Jarvis)", From 51bbc9c9ba81ba490ff64810a271545d58d1e798 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 2 Apr 2021 18:37:34 +0300 Subject: [PATCH 239/266] Update skip_list.json --- tests/queries/skip_list.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index ace00fecee0..7eff5047767 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -668,7 +668,7 @@ "01702_system_query_log", // Runs many global system queries "01715_background_checker_blather_zookeeper", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. - "01722_long_brotli_http_compression_json_format", // it is broken in some unimaginable way with the genius error "cannot write to ofstream", not sure how to debug this + "01722_long_brotli_http_compression_json_format", // it is broken in some unimaginable way with the genius error -cannot write to ofstream-, not sure how to debug this "01747_alter_partition_key_enum_zookeeper", "01748_dictionary_table_dot", // creates database "01760_polygon_dictionaries", From 514a931c2c2dcaf30b3a0b1e0830edfb866fcb20 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 2 Apr 2021 20:23:41 +0300 Subject: [PATCH 240/266] try fix skip_list parsing --- tests/queries/skip_list.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 7eff5047767..1064ddd8b4f 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -668,7 +668,7 @@ "01702_system_query_log", // Runs many global system queries "01715_background_checker_blather_zookeeper", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. - "01722_long_brotli_http_compression_json_format", // it is broken in some unimaginable way with the genius error -cannot write to ofstream-, not sure how to debug this + "01722_long_brotli_http_compression_json_format", // It is broken in some unimaginable way with the genius error 'cannot write to ofstream'. Not sure how to debug this "01747_alter_partition_key_enum_zookeeper", "01748_dictionary_table_dot", // creates database "01760_polygon_dictionaries", From 840ecb5e998203f7ee92324ad24dca644c06bef8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 2 Apr 2021 21:07:11 +0300 Subject: [PATCH 241/266] Update MergeJoin.cpp --- src/Interpreters/MergeJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 02c683b28d0..a9f50cdda0e 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -487,7 +487,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); } -/// Has to be called event totals is empty +/// Has to be called even if totals are empty void MergeJoin::setTotals(const Block & totals_block) { totals = totals_block; From fe13708cb0bb9f3dc5b1902fa5d85fa0802390c5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 2 Apr 2021 21:09:12 +0300 Subject: [PATCH 242/266] Update build-osx.md --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index e220ca612c8..886e85bbf86 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -15,7 +15,7 @@ $ /bin/bash -c "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/ ## Install Xcode and Command Line Tools {#install-xcode-and-command-line-tools} -Install the lates [Xcode](https://apps.apple.com/am/app/xcode/id497799835?mt=12) from App Store. +Install the latest [Xcode](https://apps.apple.com/am/app/xcode/id497799835?mt=12) from App Store. Open it at least once to accept the end-user license agreement and automatically install the required components. From 7a4637c6452bd8411f242c78cbf8a63a501efd98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Apr 2021 23:54:03 +0300 Subject: [PATCH 243/266] Fix deserialization of empty string without newline at end of TSV format --- src/DataTypes/Serializations/SerializationNullable.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 87805c53aa9..146aef67cd5 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -224,12 +224,9 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R { /// Little tricky, because we cannot discriminate null from first character. - if (istr.eof()) - throw ParsingException("Unexpected end of stream, while parsing value of Nullable type", ErrorCodes::CANNOT_READ_ALL_DATA); - - /// This is not null, surely. - if (*istr.position() != '\\') + if (istr.eof() || *istr.position() != '\\') /// Some data types can deserialize absense of data (e.g. empty string), so eof is ok. { + /// This is not null, surely. return safeDeserialize(column, *nested, [] { return false; }, [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); }); From 5ba6c7b7314f25140320027f81654610c71df027 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Apr 2021 00:05:40 +0300 Subject: [PATCH 244/266] FormatSettings null_as_default default value fix --- src/Formats/FormatSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c1f02c65748..33d51b1797f 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -27,7 +27,7 @@ struct FormatSettings bool with_names_use_header = false; bool write_statistics = true; bool import_nested_json = false; - bool null_as_default = false; + bool null_as_default = true; enum class DateTimeInputFormat { From 47a12a7b0b1760bf6c86565a3a3afddce9e2a891 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Apr 2021 00:14:13 +0300 Subject: [PATCH 245/266] Add a test --- .../01786_nullable_string_tsv_at_eof.reference | 6 ++++++ .../0_stateless/01786_nullable_string_tsv_at_eof.sh | 12 ++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.reference create mode 100755 tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.sh diff --git a/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.reference b/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.reference new file mode 100644 index 00000000000..35b388bbafb --- /dev/null +++ b/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.sh b/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.sh new file mode 100755 index 00000000000..f0a663ae409 --- /dev/null +++ b/tests/queries/0_stateless/01786_nullable_string_tsv_at_eof.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +printf '1\t' | $CLICKHOUSE_LOCAL --query="SELECT * FROM table" --structure='a String, b String' +printf '1\t' | $CLICKHOUSE_LOCAL --input_format_null_as_default 0 --query="SELECT * FROM table" --structure='a String, b String' +printf '1\t' | $CLICKHOUSE_LOCAL --input_format_null_as_default 1 --query="SELECT * FROM table" --structure='a String, b String' +printf '1\t' | $CLICKHOUSE_LOCAL --query="SELECT * FROM table" --structure='a String, b Nullable(String)' +printf '1\t' | $CLICKHOUSE_LOCAL --input_format_null_as_default 0 --query="SELECT * FROM table" --structure='a String, b Nullable(String)' +printf '1\t' | $CLICKHOUSE_LOCAL --input_format_null_as_default 1 --query="SELECT * FROM table" --structure='a Nullable(String), b Nullable(String)' From 07d5a1ec21ac045a0fb9bc80b03d3f0340410d56 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 2 Apr 2021 00:55:48 +0300 Subject: [PATCH 246/266] ClickHouseDictionarySource loop fix --- .../ClickHouseDictionarySource.cpp | 166 +++++++++--------- src/Dictionaries/ClickHouseDictionarySource.h | 36 ++-- ...lickhouse_dictionary_source_loop.reference | 3 + ...1780_clickhouse_dictionary_source_loop.sql | 64 +++++++ 4 files changed, 173 insertions(+), 96 deletions(-) create mode 100644 tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.reference create mode 100644 tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 5e69df2c8a0..a80ec7919a0 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -18,97 +18,72 @@ namespace DB { -static const size_t MAX_CONNECTIONS = 16; - -inline static UInt16 getPortFromContext(const Context & context, bool secure) +namespace ErrorCodes { - return secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort(); + extern const int BAD_ARGUMENTS; } -static ConnectionPoolWithFailoverPtr createPool( - const std::string & host, - UInt16 port, - bool secure, - const std::string & db, - const std::string & user, - const std::string & password) +namespace { - ConnectionPoolPtrs pools; - pools.emplace_back(std::make_shared( - MAX_CONNECTIONS, - host, - port, - db, - user, - password, - "", /* cluster */ - "", /* cluster_secret */ - "ClickHouseDictionarySource", - Protocol::Compression::Enable, - secure ? Protocol::Secure::Enable : Protocol::Secure::Disable)); - return std::make_shared(pools, LoadBalancing::RANDOM); -} + constexpr size_t MAX_CONNECTIONS = 16; + inline UInt16 getPortFromContext(const Context & context, bool secure) + { + return secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort(); + } + + ConnectionPoolWithFailoverPtr createPool(const ClickHouseDictionarySource::Configuration & configuration) + { + if (configuration.is_local) + return nullptr; + + ConnectionPoolPtrs pools; + pools.emplace_back(std::make_shared( + MAX_CONNECTIONS, + configuration.host, + configuration.port, + configuration.db, + configuration.user, + configuration.password, + "", /* cluster */ + "", /* cluster_secret */ + "ClickHouseDictionarySource", + Protocol::Compression::Enable, + configuration.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable)); + + return std::make_shared(pools, LoadBalancing::RANDOM); + } + +} ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, - const Poco::Util::AbstractConfiguration & config, - const std::string & path_to_settings, - const std::string & config_prefix, + const Configuration & configuration_, const Block & sample_block_, - const Context & context_, - const std::string & default_database) + const Context & context_) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} - , secure(config.getBool(config_prefix + ".secure", false)) - , host{config.getString(config_prefix + ".host", "localhost")} - , port(config.getInt(config_prefix + ".port", getPortFromContext(context_, secure))) - , user{config.getString(config_prefix + ".user", "default")} - , password{config.getString(config_prefix + ".password", "")} - , db{config.getString(config_prefix + ".db", default_database)} - , table{config.getString(config_prefix + ".table")} - , where{config.getString(config_prefix + ".where", "")} - , update_field{config.getString(config_prefix + ".update_field", "")} - , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} - , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} + , configuration{configuration_} + , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} - , context(context_) - , is_local{isLocalAddress({host, port}, getPortFromContext(context_, secure))} - , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} + , context{context_} + , pool{createPool(configuration)} , load_all_query{query_builder.composeLoadAllQuery()} { - /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). - if (is_local) - { - context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0)); - context = copyContextAndApplySettings(path_to_settings, context, config); - } - /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. context.makeQueryContext(); } - ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other) : update_time{other.update_time} , dict_struct{other.dict_struct} - , secure{other.secure} - , host{other.host} - , port{other.port} - , user{other.user} - , password{other.password} - , db{other.db} - , table{other.table} - , where{other.where} - , update_field{other.update_field} - , invalidate_query{other.invalidate_query} + , configuration{other.configuration} , invalidate_query_response{other.invalidate_query_response} - , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} - , context(other.context) - , is_local{other.is_local} - , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} + , context{other.context} + , pool{createPool(configuration)} , load_all_query{other.load_all_query} { context.makeQueryContext(); @@ -121,7 +96,7 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate() time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1; std::string str_time = DateLUT::instance().timeToString(hr_time); update_time = std::chrono::system_clock::now(); - return query_builder.composeUpdateQuery(update_field, str_time); + return query_builder.composeUpdateQuery(configuration.update_field, str_time); } else { @@ -155,9 +130,9 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_col bool ClickHouseDictionarySource::isModified() const { - if (!invalidate_query.empty()) + if (!configuration.invalidate_query.empty()) { - auto response = doInvalidateQuery(invalidate_query); + auto response = doInvalidateQuery(configuration.invalidate_query); LOG_TRACE(log, "Invalidate query has returned: {}, previous value: {}", response, invalidate_query_response); if (invalidate_query_response == response) return false; @@ -168,21 +143,21 @@ bool ClickHouseDictionarySource::isModified() const bool ClickHouseDictionarySource::hasUpdateField() const { - return !update_field.empty(); + return !configuration.update_field.empty(); } std::string ClickHouseDictionarySource::toString() const { - return "ClickHouse: " + db + '.' + table + (where.empty() ? "" : ", where: " + where); + const std::string & where = configuration.where; + return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); } - BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const String & query) { /// Sample block should not contain first row default values auto empty_sample_block = sample_block.cloneEmpty(); - if (is_local) + if (configuration.is_local) { auto stream = executeQuery(query, context, true).getInputStream(); stream = std::make_shared(stream, empty_sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position); @@ -195,7 +170,7 @@ BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const Strin std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const { LOG_TRACE(log, "Performing invalidate query"); - if (is_local) + if (configuration.is_local) { Context query_context = context; auto input_block = executeQuery(request, query_context, true).getInputStream(); @@ -210,7 +185,6 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re } } - void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, @@ -218,12 +192,46 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const std::string & config_prefix, Block & sample_block, const Context & context, - const std::string & default_database, + const std::string & default_database [[maybe_unused]], bool /* check_config */) -> DictionarySourcePtr { - return std::make_unique( - dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context, default_database); + bool secure = config.getBool(config_prefix + ".secure", false); + Context context_copy = context; + + UInt16 default_port = getPortFromContext(context_copy, secure); + + std::string settings_config_prefix = config_prefix + ".clickhouse"; + + ClickHouseDictionarySource::Configuration configuration { + .secure = config.getBool(settings_config_prefix + ".secure", false), + .host = config.getString(settings_config_prefix + ".host", "localhost"), + .port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)), + .user = config.getString(settings_config_prefix + ".user", "default"), + .password = config.getString(settings_config_prefix + ".password", ""), + .db = config.getString(settings_config_prefix + ".db", default_database), + .table = config.getString(settings_config_prefix + ".table"), + .where = config.getString(settings_config_prefix + ".where", ""), + .update_field = config.getString(settings_config_prefix + ".update_field", ""), + .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), + .is_local = isLocalAddress({configuration.host, configuration.port}, default_port) + }; + + /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). + if (configuration.is_local) + { + context_copy.setUser(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); + context_copy = copyContextAndApplySettings(config_prefix, context_copy, config); + } + + String dictionary_name = config.getString(".dictionary.name", ""); + String dictionary_database = config.getString(".dictionary.database", ""); + + if (dictionary_name == configuration.table && dictionary_database == configuration.db) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); + + return std::make_unique(dict_struct, configuration, sample_block, context_copy); }; + factory.registerSource("clickhouse", create_table_source); } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 4f26dd16849..ac615848b49 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -18,14 +18,26 @@ namespace DB class ClickHouseDictionarySource final : public IDictionarySource { public: + struct Configuration + { + const bool secure; + const std::string host; + const UInt16 port; + const std::string user; + const std::string password; + const std::string db; + const std::string table; + const std::string where; + const std::string update_field; + const std::string invalidate_query; + const bool is_local; + }; + ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, - const Poco::Util::AbstractConfiguration & config, - const std::string & path_to_settings, - const std::string & config_prefix, + const Configuration & configuration_, const Block & sample_block_, - const Context & context, - const std::string & default_database); + const Context & context); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); @@ -50,7 +62,7 @@ public: /// Used for detection whether the hashtable should be preallocated /// (since if there is WHERE then it can filter out too much) - bool hasWhere() const { return !where.empty(); } + bool hasWhere() const { return !configuration.where.empty(); } private: std::string getUpdateFieldAndDate(); @@ -61,21 +73,11 @@ private: std::chrono::time_point update_time; const DictionaryStructure dict_struct; - const bool secure; - const std::string host; - const UInt16 port; - const std::string user; - const std::string password; - const std::string db; - const std::string table; - const std::string where; - const std::string update_field; - std::string invalidate_query; + const Configuration configuration; mutable std::string invalidate_query_response; ExternalQueryBuilder query_builder; Block sample_block; Context context; - const bool is_local; ConnectionPoolWithFailoverPtr pool; const std::string load_all_query; Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource"); diff --git a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.reference b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.reference new file mode 100644 index 00000000000..0cfb83aa2f2 --- /dev/null +++ b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.reference @@ -0,0 +1,3 @@ +1 1 +2 2 +3 3 diff --git a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql new file mode 100644 index 00000000000..95c604cc01f --- /dev/null +++ b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql @@ -0,0 +1,64 @@ +DROP DICTIONARY IF EXISTS dict1; +CREATE DICTIONARY dict1 +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict1')) +LAYOUT(DIRECT()); + +SELECT * FROM dict1; --{serverError 36} + +DROP DICTIONARY dict1; + +DROP DICTIONARY IF EXISTS dict2; +CREATE DICTIONARY default.dict2 +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict2')) +LAYOUT(DIRECT()); + +SELECT * FROM dict2; --{serverError 36} + +DROP DICTIONARY dict2; + +DROP DATABASE IF EXISTS 01780_db; +CREATE DATABASE 01780_db; + +DROP DICTIONARY IF EXISTS dict3; +CREATE DICTIONARY 01780_db.dict3 +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict3')) +LAYOUT(DIRECT()); + +SELECT * FROM 01780_db.dict3; --{serverError 36} +DROP DICTIONARY 01780_db.dict3; + +CREATE TABLE 01780_db.dict3_source +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO 01780_db.dict3_source VALUES (1, '1'), (2, '2'), (3, '3'); + +CREATE DICTIONARY 01780_db.dict3 +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict3_source' DATABASE '01780_db')) +LAYOUT(DIRECT()); + +SELECT * FROM 01780_db.dict3; + +DROP DICTIONARY 01780_db.dict3; From d84fd76f3f5b87c2a7ebb5b587ca526a2b8f50e8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Apr 2021 00:39:13 +0300 Subject: [PATCH 247/266] Fixed test --- ...1780_clickhouse_dictionary_source_loop.sql | 26 ++++--------------- tests/queries/skip_list.json | 3 ++- 2 files changed, 7 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql index 95c604cc01f..53c6fe493e9 100644 --- a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql +++ b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql @@ -13,35 +13,19 @@ SELECT * FROM dict1; --{serverError 36} DROP DICTIONARY dict1; DROP DICTIONARY IF EXISTS dict2; -CREATE DICTIONARY default.dict2 +CREATE DICTIONARY 01780_db.dict2 ( id UInt64, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict2')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict2')) LAYOUT(DIRECT()); -SELECT * FROM dict2; --{serverError 36} - -DROP DICTIONARY dict2; - -DROP DATABASE IF EXISTS 01780_db; -CREATE DATABASE 01780_db; - -DROP DICTIONARY IF EXISTS dict3; -CREATE DICTIONARY 01780_db.dict3 -( - id UInt64, - value String -) -PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict3')) -LAYOUT(DIRECT()); - -SELECT * FROM 01780_db.dict3; --{serverError 36} -DROP DICTIONARY 01780_db.dict3; +SELECT * FROM 01780_db.dict2; --{serverError 36} +DROP DICTIONARY 01780_db.dict2; +DROP TABLE IF EXISTS 01780_db.dict3_source; CREATE TABLE 01780_db.dict3_source ( id UInt64, diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 1064ddd8b4f..af3081241e8 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -694,6 +694,7 @@ "01685_ssd_cache_dictionary_complex_key", "01760_system_dictionaries", "01760_polygon_dictionaries", - "01778_hierarchical_dictionaries" + "01778_hierarchical_dictionaries", + "01780_clickhouse_dictionary_source_loop" ] } From 9c93ce7ff68a55ab0c41237967f038b88cae8939 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Apr 2021 01:57:04 +0300 Subject: [PATCH 248/266] Minor fixes in tests --- .../0_stateless/00120_join_and_group_by.reference | 10 +++++----- tests/queries/0_stateless/00120_join_and_group_by.sql | 2 +- .../0_stateless/00206_empty_array_to_single.sql | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00120_join_and_group_by.reference b/tests/queries/0_stateless/00120_join_and_group_by.reference index 417dbb5b661..9b3afb80a8d 100644 --- a/tests/queries/0_stateless/00120_join_and_group_by.reference +++ b/tests/queries/0_stateless/00120_join_and_group_by.reference @@ -1,11 +1,11 @@ 0 +4416930539393268817 1241149650 9 +4761183170873013810 4249604106 0 7766709361750702608 3902320246 4 +9624464864560415994 1298551497 3 10577349846663553072 1343103100 1 11700034558374135620 1618865725 8 -18198135717204167749 1996614413 2 -4761183170873013810 4249604106 0 12742043333840853032 1295823179 6 -9624464864560415994 1298551497 3 -4416930539393268817 1241149650 9 -15228578409069794350 2641603337 5 13365811232860260488 3844986530 7 +15228578409069794350 2641603337 5 +18198135717204167749 1996614413 2 diff --git a/tests/queries/0_stateless/00120_join_and_group_by.sql b/tests/queries/0_stateless/00120_join_and_group_by.sql index 21168443ee4..005e25a7127 100644 --- a/tests/queries/0_stateless/00120_join_and_group_by.sql +++ b/tests/queries/0_stateless/00120_join_and_group_by.sql @@ -3,4 +3,4 @@ SELECT value FROM system.one ANY LEFT JOIN (SELECT dummy, dummy AS value) js2 US SELECT value1, value2, sum(number) FROM (SELECT number, intHash64(number) AS value1 FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN (SELECT number, intHash32(number) AS value2 FROM system.numbers LIMIT 10) js2 -USING number GROUP BY value1, value2; +USING number GROUP BY value1, value2 ORDER BY value1, value2; diff --git a/tests/queries/0_stateless/00206_empty_array_to_single.sql b/tests/queries/0_stateless/00206_empty_array_to_single.sql index 9724aa3fda7..0e3ff4f3537 100644 --- a/tests/queries/0_stateless/00206_empty_array_to_single.sql +++ b/tests/queries/0_stateless/00206_empty_array_to_single.sql @@ -1,5 +1,5 @@ SELECT emptyArrayToSingle(arrayFilter(x -> x != 99, arrayJoin([[1, 2], [99], [4, 5, 6]]))); -SELECT emptyArrayToSingle(emptyArrayString()), emptyArrayToSingle(emptyArrayDate()), emptyArrayToSingle(emptyArrayDateTime()); +SELECT emptyArrayToSingle(emptyArrayString()), emptyArrayToSingle(emptyArrayDate()), emptyArrayToSingle(arrayFilter(x -> 0, [now('Europe/Moscow')])); SELECT emptyArrayToSingle(range(number % 3)), From 2e8e116d4ccce7e000ab20d542b44b8d3b98ffed Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Apr 2021 12:26:09 +0300 Subject: [PATCH 249/266] Filter removed/renamed tests from ci-changed-files.txt for fuzzer --- docker/test/fuzzer/run-fuzzer.sh | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index e21f9efae66..4bd3fa717a2 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -69,11 +69,25 @@ function watchdog killall -9 clickhouse-client ||: } +function filter_exists +{ + local path + for path in "$@"; do + if [ -e "$path" ]; then + echo "$path" + else + echo "'$path' does not exists" >&2 + fi + done +} + function fuzz { # Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests. # Don't overwrite the NEW_TESTS_OPT so that it can be set from the environment. NEW_TESTS="$(grep -P 'tests/queries/0_stateless/.*\.sql' ci-changed-files.txt | sed -r -e 's!^!ch/!' | sort -R)" + # ci-changed-files.txt contains also files that has been deleted/renamed, filter them out. + NEW_TESTS="$(filter_exists $NEW_TESTS)" if [[ -n "$NEW_TESTS" ]] then NEW_TESTS_OPT="${NEW_TESTS_OPT:---interleave-queries-file ${NEW_TESTS}}" From 2de808ec5acc6d306d7470f4bacc8b3cd19ce578 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Apr 2021 12:50:57 +0300 Subject: [PATCH 250/266] Update SerializationNullable.cpp --- src/DataTypes/Serializations/SerializationNullable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 146aef67cd5..4de2b08c043 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -224,7 +224,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R { /// Little tricky, because we cannot discriminate null from first character. - if (istr.eof() || *istr.position() != '\\') /// Some data types can deserialize absense of data (e.g. empty string), so eof is ok. + if (istr.eof() || *istr.position() != '\\') /// Some data types can deserialize absence of data (e.g. empty string), so eof is ok. { /// This is not null, surely. return safeDeserialize(column, *nested, From 0733144b4a355394fecb837ba01af76a50e3fa05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Apr 2021 12:53:57 +0300 Subject: [PATCH 251/266] Lower test scale --- ...01746_long_zlib_http_compression_json_format.reference | 8 ++++---- .../01746_long_zlib_http_compression_json_format.sh | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference index 7c089a2fd05..92dfd99c259 100644 --- a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference +++ b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference @@ -5,7 +5,7 @@ "host": "clickhouse-test-host-001.clickhouse.com", "home": "clickhouse", "detail": "clickhouse", - "row_number": "999998" + "row_number": "99998" }, { "datetime": "2020-12-12", @@ -13,11 +13,11 @@ "host": "clickhouse-test-host-001.clickhouse.com", "home": "clickhouse", "detail": "clickhouse", - "row_number": "999999" + "row_number": "99999" } ], - "rows": 1000000, + "rows": 100000, - "rows_before_limit_at_least": 1048080, + "rows_before_limit_at_least": 131010, diff --git a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh index e663b329660..7a2343a953a 100755 --- a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh +++ b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | gzip -d | tail -n30 | head -n23 +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23 From 541b601317f0b1e55e9a18492858cd218963eda8 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Sat, 3 Apr 2021 13:25:40 +0300 Subject: [PATCH 252/266] Try fix flaky test --- tests/integration/test_attach_without_fetching/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index a79a7babc08..8b0c1ffbc5c 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -122,8 +122,8 @@ def test_attach_without_fetching(start_cluster): with PartitionManager() as pm: # If something goes wrong and replica 2 wants to fetch data, the test will fail. - pm.partition_instances(node_2, node_1) - pm.partition_instances(node_1, node_3) + pm.partition_instances(node_2, node_1, action='REJECT --reject-with tcp-reset') + pm.partition_instances(node_1, node_3, action='REJECT --reject-with tcp-reset') node_1.query("ALTER TABLE test ATTACH PART '0_0_0_0'") From 2f1f9692e6baab83db6a23779fe835ad2f072c7f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 3 Apr 2021 13:58:11 +0300 Subject: [PATCH 253/266] Fixed test --- .../0_stateless/01780_clickhouse_dictionary_source_loop.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql index 53c6fe493e9..5673e646a47 100644 --- a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql +++ b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql @@ -1,3 +1,6 @@ +DROP DATABASE IF EXISTS 01780_db; +CREATE DATABASE 01780_db; + DROP DICTIONARY IF EXISTS dict1; CREATE DICTIONARY dict1 ( @@ -46,3 +49,5 @@ LAYOUT(DIRECT()); SELECT * FROM 01780_db.dict3; DROP DICTIONARY 01780_db.dict3; + +DROP DATABASE 01780_db; From 10d35867e3efe60b72142d9d6c4aff03a43f5a96 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Apr 2021 16:03:17 +0300 Subject: [PATCH 254/266] Drop mutable qualifier for StorageTinyLog::rwlock --- src/Storages/StorageTinyLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index b76e8e34dfb..1187f7f905d 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -70,7 +70,7 @@ private: Files files; FileChecker file_checker; - mutable std::shared_timed_mutex rwlock; + std::shared_timed_mutex rwlock; Poco::Logger * log; From 6cebd3ee1f6af71cd31da2489ecb369c7268c391 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Apr 2021 16:03:37 +0300 Subject: [PATCH 255/266] Fix UB by unlocking the rwlock of the TinyLog from the same thread Before this patch the build with libstdc++ hangs for 00967_insert_into_distributed_different_types test (and I guess some others), this is because the rwlock is acquired from different thread as it was unlocked which causes UB [1], fix this by moving unlock into writeSuffix(). [1]: The pthread_rwlock_unlock() function shall release a lock held on the read-write lock object referenced by rwlock. Results are undefined if the read-write lock rwlock is not held by the **calling thread**. --- src/Storages/StorageTinyLog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 3cb4be50a86..6ce41dac614 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -357,6 +357,8 @@ void TinyLogBlockOutputStream::writeSuffix() for (const auto & file : column_files) storage.file_checker.update(file); storage.file_checker.save(); + + lock.unlock(); } From 36b03b27401afcf0b0ee704c827a1f89d377e50e Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 3 Apr 2021 22:17:51 +0400 Subject: [PATCH 256/266] Compilation fix --- src/Dictionaries/HashedDictionary.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d45e4ade1cf..3fbab4c3e39 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -200,7 +200,7 @@ ColumnPtr HashedDictionary::getHierarchy(ColumnPtr const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; - const UInt64 null_value = dictionary_attribute.null_value.get(); + const UInt64 null_value = dictionary_attribute.null_value.template get(); const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); }; @@ -246,7 +246,7 @@ ColumnUInt8::Ptr HashedDictionary::isInHierarchy( const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; - const UInt64 null_value = dictionary_attribute.null_value.get(); + const UInt64 null_value = dictionary_attribute.null_value.template get(); const CollectionType & parent_keys_map = std::get>(hierarchical_attribute.container); auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); }; @@ -327,14 +327,14 @@ void HashedDictionary::createAttributes() { string_arena = std::make_unique(); - const auto & string_null_value = dictionary_attribute.null_value.get(); + const auto & string_null_value = dictionary_attribute.null_value.template get(); const size_t string_null_value_size = string_null_value.size(); const char * string_in_arena = string_arena->insert(string_null_value.data(), string_null_value_size); default_value = {string_in_arena, string_null_value_size}; } else - default_value = dictionary_attribute.null_value.get>(); + default_value = dictionary_attribute.null_value.template get>(); Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), default_value, CollectionType(), std::move(string_arena)}; attributes.emplace_back(std::move(attribute)); From 305048630874e5169d000f6e47576d0e9dbef727 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Apr 2021 22:50:49 +0300 Subject: [PATCH 257/266] Fix assert in Arena --- src/Columns/IColumnDummy.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index e844b54aaed..375c4f259d9 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -59,13 +59,14 @@ public: StringRef serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin) const override { - return { arena.allocContinue(0, begin), 0 }; + /// Has to put one useless byte into Arena, because serialization into zero bytes is ambiguous. + return { arena.allocContinue(1, begin), 1 }; } const char * deserializeAndInsertFromArena(const char * pos) override { ++s; - return pos; + return pos + 1; } const char * skipSerializedInArena(const char * pos) const override From e684b463aedb46068ea376a46e7a30cc1bcbf245 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Apr 2021 22:51:37 +0300 Subject: [PATCH 258/266] Add test --- .../0_stateless/01787_arena_assert_column_nothing.reference | 1 + tests/queries/0_stateless/01787_arena_assert_column_nothing.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01787_arena_assert_column_nothing.reference create mode 100644 tests/queries/0_stateless/01787_arena_assert_column_nothing.sql diff --git a/tests/queries/0_stateless/01787_arena_assert_column_nothing.reference b/tests/queries/0_stateless/01787_arena_assert_column_nothing.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01787_arena_assert_column_nothing.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01787_arena_assert_column_nothing.sql b/tests/queries/0_stateless/01787_arena_assert_column_nothing.sql new file mode 100644 index 00000000000..de6374a1bc3 --- /dev/null +++ b/tests/queries/0_stateless/01787_arena_assert_column_nothing.sql @@ -0,0 +1 @@ +SELECT 1 GROUP BY emptyArrayToSingle(arrayFilter(x -> 1, [])); From 65ca073f42a8cc4ab16f9d95d4ae3174d5e93e32 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Sun, 4 Apr 2021 01:05:07 +0300 Subject: [PATCH 259/266] Allow unbundled xz (lzma) using USE_INTERNAL_XZ_LIBRARY=OFF --- CMakeLists.txt | 1 + cmake/find/xz.cmake | 27 +++++++++++++++++++++++++++ src/CMakeLists.txt | 8 +++----- 3 files changed, 31 insertions(+), 5 deletions(-) create mode 100644 cmake/find/xz.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 5d716985c46..c4d429c565f 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -464,6 +464,7 @@ find_contrib_lib(double-conversion) # Must be before parquet include (cmake/find/ssl.cmake) include (cmake/find/ldap.cmake) # after ssl include (cmake/find/icu.cmake) +include (cmake/find/xz.cmake) include (cmake/find/zlib.cmake) include (cmake/find/zstd.cmake) include (cmake/find/ltdl.cmake) # for odbc diff --git a/cmake/find/xz.cmake b/cmake/find/xz.cmake new file mode 100644 index 00000000000..ef6ee3e9244 --- /dev/null +++ b/cmake/find/xz.cmake @@ -0,0 +1,27 @@ +option (USE_INTERNAL_XZ_LIBRARY "Set to OFF to use system xz (lzma) library instead of bundled" ${NOT_UNBUNDLED}) + +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api/lzma.h") + if(USE_INTERNAL_XZ_LIBRARY) + message(WARNING "submodule contrib/xz is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal xz (lzma) library") + set(USE_INTERNAL_XZ_LIBRARY 0) + endif() + set(MISSING_INTERNAL_XZ_LIBRARY 1) +endif() + +if (NOT USE_INTERNAL_XZ_LIBRARY) + find_library (XZ_LIBRARY lzma) + find_path (XZ_INCLUDE_DIR NAMES lzma.h PATHS ${XZ_INCLUDE_PATHS}) + if (NOT XZ_LIBRARY OR NOT XZ_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system xz (lzma) library") + endif () +endif () + +if (XZ_LIBRARY AND XZ_INCLUDE_DIR) +elseif (NOT MISSING_INTERNAL_XZ_LIBRARY) + set (USE_INTERNAL_XZ_LIBRARY 1) + set (XZ_LIBRARY liblzma) + set (XZ_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api) +endif () + +message (STATUS "Using xz (lzma): ${XZ_INCLUDE_DIR} : ${XZ_LIBRARY}") diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fe519530123..a41ddada5a9 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -375,11 +375,9 @@ if (ZSTD_LIBRARY) endif () endif() -set (LZMA_LIBRARY liblzma) -set (LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api) -if (LZMA_LIBRARY) - target_link_libraries (clickhouse_common_io PUBLIC ${LZMA_LIBRARY}) - target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR}) +if (XZ_LIBRARY) + target_link_libraries (clickhouse_common_io PUBLIC ${XZ_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${XZ_INCLUDE_DIR}) endif() if (USE_ICU) From 4a79c52978e4921390ffe25fbe2ad090bcb791b8 Mon Sep 17 00:00:00 2001 From: mastertheknife Date: Sun, 4 Apr 2021 01:16:00 +0300 Subject: [PATCH 260/266] Fix tab\space in xz.cmake --- cmake/find/xz.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/xz.cmake b/cmake/find/xz.cmake index ef6ee3e9244..0d19859c6b1 100644 --- a/cmake/find/xz.cmake +++ b/cmake/find/xz.cmake @@ -3,7 +3,7 @@ option (USE_INTERNAL_XZ_LIBRARY "Set to OFF to use system xz (lzma) library inst if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api/lzma.h") if(USE_INTERNAL_XZ_LIBRARY) message(WARNING "submodule contrib/xz is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal xz (lzma) library") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal xz (lzma) library") set(USE_INTERNAL_XZ_LIBRARY 0) endif() set(MISSING_INTERNAL_XZ_LIBRARY 1) From b9e8ff137aee1f1a172e527a0f4846eab4ff2e44 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 4 Apr 2021 01:19:57 +0300 Subject: [PATCH 261/266] Fixed build --- src/Dictionaries/ClickHouseDictionarySource.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index a80ec7919a0..d89271932eb 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -201,11 +201,13 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) UInt16 default_port = getPortFromContext(context_copy, secure); std::string settings_config_prefix = config_prefix + ".clickhouse"; + std::string host = config.getString(settings_config_prefix + ".host", "localhost"); + UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); ClickHouseDictionarySource::Configuration configuration { .secure = config.getBool(settings_config_prefix + ".secure", false), - .host = config.getString(settings_config_prefix + ".host", "localhost"), - .port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)), + .host = host, + .port = port, .user = config.getString(settings_config_prefix + ".user", "default"), .password = config.getString(settings_config_prefix + ".password", ""), .db = config.getString(settings_config_prefix + ".db", default_database), @@ -213,7 +215,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .where = config.getString(settings_config_prefix + ".where", ""), .update_field = config.getString(settings_config_prefix + ".update_field", ""), .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), - .is_local = isLocalAddress({configuration.host, configuration.port}, default_port) + .is_local = isLocalAddress({host, port}, default_port) }; /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). From 3f2f53d2bb1f695fb9c0626efcdebe077eec2610 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Apr 2021 04:52:41 +0300 Subject: [PATCH 262/266] Allow query profiling only on x86_64 --- programs/server/Server.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0aaf47989b9..f2f43aabc7d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -986,7 +986,7 @@ int Server::main(const std::vector & /*args*/) /// /// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h /// -#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER) +#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER) && defined(__x86_64__) /// Profilers cannot work reliably with any other libunwind or without PHDR cache. if (hasPHDRCache()) { @@ -1023,6 +1023,10 @@ int Server::main(const std::vector & /*args*/) " when two different stack unwinding methods will interfere with each other."); #endif +#if !defined(__x86_64__) + LOG_INFO(log, "Query Profiler is only tested on x86_64. It also known to not work under qemu-user."); +#endif + if (!hasPHDRCache()) LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); From b35ba1152fe1f8b4e69afbad2eeea6b687a4b4b2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 4 Apr 2021 05:04:11 +0300 Subject: [PATCH 263/266] Update IColumnDummy.h --- src/Columns/IColumnDummy.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 375c4f259d9..7e1958f077e 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -59,8 +59,10 @@ public: StringRef serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin) const override { - /// Has to put one useless byte into Arena, because serialization into zero bytes is ambiguous. - return { arena.allocContinue(1, begin), 1 }; + /// Has to put one useless byte into Arena, because serialization into zero number of bytes is ambiguous. + char * res = arena.allocContinue(1, begin); + *res = 0; + return { res, 1 }; } const char * deserializeAndInsertFromArena(const char * pos) override From a0f2b85cd3e0bbaf0ac5f5ffb95b0d7d3ab2dfbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Apr 2021 05:21:56 +0300 Subject: [PATCH 264/266] Make round function to behave consistently on non-x86_64 --- src/Common/ErrorCodes.cpp | 1 + src/Functions/FunctionsRound.h | 16 ++++++++++++++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 918bc301754..0f85ad5c792 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -547,6 +547,7 @@ M(577, INVALID_SHARD_ID) \ M(578, INVALID_FORMAT_INSERT_QUERY_WITH_DATA) \ M(579, INCORRECT_PART_TYPE) \ + M(580, CANNOT_SET_ROUNDING_MODE) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index b510f62662e..b190bc5bd8b 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -21,6 +21,8 @@ #ifdef __SSE4_1__ #include +#else + #include #endif @@ -34,6 +36,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; + extern const int CANNOT_SET_ROUNDING_MODE; } @@ -231,7 +234,7 @@ inline float roundWithMode(float x, RoundingMode mode) { switch (mode) { - case RoundingMode::Round: return roundf(x); + case RoundingMode::Round: return nearbyintf(x); case RoundingMode::Floor: return floorf(x); case RoundingMode::Ceil: return ceilf(x); case RoundingMode::Trunc: return truncf(x); @@ -244,7 +247,7 @@ inline double roundWithMode(double x, RoundingMode mode) { switch (mode) { - case RoundingMode::Round: return round(x); + case RoundingMode::Round: return nearbyint(x); case RoundingMode::Floor: return floor(x); case RoundingMode::Ceil: return ceil(x); case RoundingMode::Trunc: return trunc(x); @@ -595,6 +598,15 @@ public: return false; }; +#if !defined(__SSE4_1__) + /// In case of "nearbyint" function is used, we should ensure the expected rounding mode for the Banker's rounding. + /// Actually it is by default. But we will set it just in case. + + if constexpr (rounding_mode == RoundingMode::Round) + if (0 != fesetround(FE_TONEAREST)) + throw Exception("Cannot set floating point rounding mode", ErrorCodes::CANNOT_SET_ROUNDING_MODE); +#endif + if (!callOnIndexAndDataType(column.type->getTypeId(), call)) { throw Exception("Illegal column " + column.name + " of argument of function " + getName(), From 41dfcb7bece6ddd2489846917acbeef2066bd69d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Apr 2021 12:24:00 +0300 Subject: [PATCH 265/266] Fix flaky test after #22427 --- .../0_stateless/01263_type_conversion_nvartolomei.reference | 2 -- tests/queries/0_stateless/01263_type_conversion_nvartolomei.sql | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01263_type_conversion_nvartolomei.reference b/tests/queries/0_stateless/01263_type_conversion_nvartolomei.reference index 09b593dad3d..97c766822ac 100644 --- a/tests/queries/0_stateless/01263_type_conversion_nvartolomei.reference +++ b/tests/queries/0_stateless/01263_type_conversion_nvartolomei.reference @@ -3,5 +3,3 @@ a a --- -a -a diff --git a/tests/queries/0_stateless/01263_type_conversion_nvartolomei.sql b/tests/queries/0_stateless/01263_type_conversion_nvartolomei.sql index e3d66e9cdba..0eeb97e2b2d 100644 --- a/tests/queries/0_stateless/01263_type_conversion_nvartolomei.sql +++ b/tests/queries/0_stateless/01263_type_conversion_nvartolomei.sql @@ -43,7 +43,7 @@ SELECT * FROM d; SELECT '---'; INSERT INTO m VALUES ('b'); -SELECT v FROM d ORDER BY v; -- { clientError 36 } +SELECT toString(v) FROM (SELECT v FROM d ORDER BY v) FORMAT Null; -- { serverError 36 } DROP TABLE m; From 3ae0c7e17afa2d93e2b772fdc56fa5e0d2ee35e9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Apr 2021 12:55:04 +0300 Subject: [PATCH 266/266] Fix CPU-dependent behaviour in tests --- .../00232_format_readable_size.reference | 96 +++++++++---------- .../00232_format_readable_size.sql | 2 +- 2 files changed, 49 insertions(+), 49 deletions(-) diff --git a/tests/queries/0_stateless/00232_format_readable_size.reference b/tests/queries/0_stateless/00232_format_readable_size.reference index 0f723e968d9..b9b9b467b50 100644 --- a/tests/queries/0_stateless/00232_format_readable_size.reference +++ b/tests/queries/0_stateless/00232_format_readable_size.reference @@ -20,51 +20,51 @@ 170.21 MiB 170.21 MiB 170.21 MiB 462.69 MiB 462.69 MiB 462.69 MiB 1.23 GiB 1.23 GiB 1.23 GiB -3.34 GiB 3.34 GiB -2.00 GiB -9.08 GiB 9.08 GiB -2.00 GiB -24.67 GiB 24.67 GiB -2.00 GiB -67.06 GiB 67.06 GiB -2.00 GiB -182.29 GiB 182.29 GiB -2.00 GiB -495.51 GiB 495.51 GiB -2.00 GiB -1.32 TiB 1.32 TiB -2.00 GiB -3.58 TiB 3.58 TiB -2.00 GiB -9.72 TiB 9.72 TiB -2.00 GiB -26.42 TiB 26.42 TiB -2.00 GiB -71.82 TiB 71.82 TiB -2.00 GiB -195.22 TiB 195.22 TiB -2.00 GiB -530.66 TiB 530.66 TiB -2.00 GiB -1.41 PiB 1.41 PiB -2.00 GiB -3.83 PiB 3.83 PiB -2.00 GiB -10.41 PiB 10.41 PiB -2.00 GiB -28.29 PiB 28.29 PiB -2.00 GiB -76.91 PiB 76.91 PiB -2.00 GiB -209.06 PiB 209.06 PiB -2.00 GiB -568.30 PiB 568.30 PiB -2.00 GiB -1.51 EiB 1.51 EiB -2.00 GiB -4.10 EiB 4.10 EiB -2.00 GiB -11.15 EiB 11.15 EiB -2.00 GiB -30.30 EiB 0.00 B -2.00 GiB -82.37 EiB 0.00 B -2.00 GiB -223.89 EiB 0.00 B -2.00 GiB -608.60 EiB 0.00 B -2.00 GiB -1.62 ZiB 0.00 B -2.00 GiB -4.39 ZiB 0.00 B -2.00 GiB -11.94 ZiB 0.00 B -2.00 GiB -32.45 ZiB 0.00 B -2.00 GiB -88.21 ZiB 0.00 B -2.00 GiB -239.77 ZiB 0.00 B -2.00 GiB -651.77 ZiB 0.00 B -2.00 GiB -1.73 YiB 0.00 B -2.00 GiB -4.70 YiB 0.00 B -2.00 GiB -12.78 YiB 0.00 B -2.00 GiB -34.75 YiB 0.00 B -2.00 GiB -94.46 YiB 0.00 B -2.00 GiB -256.78 YiB 0.00 B -2.00 GiB -698.00 YiB 0.00 B -2.00 GiB -1897.37 YiB 0.00 B -2.00 GiB -5157.59 YiB 0.00 B -2.00 GiB -14019.80 YiB 0.00 B -2.00 GiB -38109.75 YiB 0.00 B -2.00 GiB -103593.05 YiB 0.00 B -2.00 GiB -281595.11 YiB 0.00 B -2.00 GiB -765454.88 YiB 0.00 B -2.00 GiB +3.34 GiB 3.34 GiB 2.00 GiB +9.08 GiB 9.08 GiB 2.00 GiB +24.67 GiB 24.67 GiB 2.00 GiB +67.06 GiB 67.06 GiB 2.00 GiB +182.29 GiB 182.29 GiB 2.00 GiB +495.51 GiB 495.51 GiB 2.00 GiB +1.32 TiB 1.32 TiB 2.00 GiB +3.58 TiB 3.58 TiB 2.00 GiB +9.72 TiB 9.72 TiB 2.00 GiB +26.42 TiB 26.42 TiB 2.00 GiB +71.82 TiB 71.82 TiB 2.00 GiB +195.22 TiB 195.22 TiB 2.00 GiB +530.66 TiB 530.66 TiB 2.00 GiB +1.41 PiB 1.41 PiB 2.00 GiB +3.83 PiB 3.83 PiB 2.00 GiB +10.41 PiB 10.41 PiB 2.00 GiB +28.29 PiB 28.29 PiB 2.00 GiB +76.91 PiB 76.91 PiB 2.00 GiB +209.06 PiB 209.06 PiB 2.00 GiB +568.30 PiB 568.30 PiB 2.00 GiB +1.51 EiB 1.51 EiB 2.00 GiB +4.10 EiB 4.10 EiB 2.00 GiB +11.15 EiB 11.15 EiB 2.00 GiB +30.30 EiB 16.00 EiB 2.00 GiB +82.37 EiB 16.00 EiB 2.00 GiB +223.89 EiB 16.00 EiB 2.00 GiB +608.60 EiB 16.00 EiB 2.00 GiB +1.62 ZiB 16.00 EiB 2.00 GiB +4.39 ZiB 16.00 EiB 2.00 GiB +11.94 ZiB 16.00 EiB 2.00 GiB +32.45 ZiB 16.00 EiB 2.00 GiB +88.21 ZiB 16.00 EiB 2.00 GiB +239.77 ZiB 16.00 EiB 2.00 GiB +651.77 ZiB 16.00 EiB 2.00 GiB +1.73 YiB 16.00 EiB 2.00 GiB +4.70 YiB 16.00 EiB 2.00 GiB +12.78 YiB 16.00 EiB 2.00 GiB +34.75 YiB 16.00 EiB 2.00 GiB +94.46 YiB 16.00 EiB 2.00 GiB +256.78 YiB 16.00 EiB 2.00 GiB +698.00 YiB 16.00 EiB 2.00 GiB +1897.37 YiB 16.00 EiB 2.00 GiB +5157.59 YiB 16.00 EiB 2.00 GiB +14019.80 YiB 16.00 EiB 2.00 GiB +38109.75 YiB 16.00 EiB 2.00 GiB +103593.05 YiB 16.00 EiB 2.00 GiB +281595.11 YiB 16.00 EiB 2.00 GiB +765454.88 YiB 16.00 EiB 2.00 GiB diff --git a/tests/queries/0_stateless/00232_format_readable_size.sql b/tests/queries/0_stateless/00232_format_readable_size.sql index 952ee82b81a..e96f7ebeb20 100644 --- a/tests/queries/0_stateless/00232_format_readable_size.sql +++ b/tests/queries/0_stateless/00232_format_readable_size.sql @@ -1,4 +1,4 @@ -WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z +WITH round(exp(number), 6) AS x, x > 0xFFFFFFFFFFFFFFFF ? 0xFFFFFFFFFFFFFFFF : toUInt64(x) AS y, x > 0x7FFFFFFF ? 0x7FFFFFFF : toInt32(x) AS z SELECT formatReadableSize(x), formatReadableSize(y), formatReadableSize(z) FROM system.numbers LIMIT 70;