From 1c41d73d9b232d674e1a64524d154a03ccc55099 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 10 Feb 2020 12:27:57 +0300 Subject: [PATCH 001/147] small refactoring + comments adding --- dbms/programs/copier/Internals.h | 8 ++++++++ dbms/src/Parsers/ASTLiteral.h | 4 ++-- 2 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 dbms/programs/copier/Internals.h diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h new file mode 100644 index 00000000000..a25ae7b973c --- /dev/null +++ b/dbms/programs/copier/Internals.h @@ -0,0 +1,8 @@ +// +// Created by jakalletti on 2/7/20. +// + +#ifndef CLICKHOUSE_INTERNALS_H +#define CLICKHOUSE_INTERNALS_H + +#endif //CLICKHOUSE_INTERNALS_H diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 552f5da04a2..1d307a4101e 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -21,8 +21,8 @@ public: std::optional begin; std::optional end; - ASTLiteral(Field && value_) : value(value_) {} - ASTLiteral(const Field & value_) : value(value_) {} + explicit ASTLiteral(Field && value_) : value(value_) {} + explicit ASTLiteral(const Field & value_) : value(value_) {} /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } From b481682a1e067d44219bd47e752f685f1e704513 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 11 Feb 2020 14:19:19 +0300 Subject: [PATCH 002/147] just comments --- dbms/programs/copier/ClusterCopier.cpp | 872 ++++--------------------- dbms/programs/copier/ClusterCopier.h | 4 +- dbms/programs/copier/Internals.h | 750 ++++++++++++++++++++- 3 files changed, 854 insertions(+), 772 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 2c6b16a7ae4..e8b3018a861 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,87 +1,10 @@ #include "ClusterCopier.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - +#include "Internals.h" namespace DB { -namespace ErrorCodes -{ - extern const int NO_ZOOKEEPER; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TABLE; - extern const int UNFINISHED; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; -} - using ConfigurationPtr = Poco::AutoPtr; @@ -92,661 +15,19 @@ static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_da return {new Poco::Util::XMLConfiguration{&input_source}}; } -namespace -{ - - -using DatabaseAndTableName = std::pair; - -String getQuotedTable(const String & database, const String & table) -{ - if (database.empty()) - { - return backQuoteIfNeed(table); - } - - return backQuoteIfNeed(database) + "." + backQuoteIfNeed(table); -} - -String getQuotedTable(const DatabaseAndTableName & db_and_table) -{ - return getQuotedTable(db_and_table.first, db_and_table.second); -} - - -enum class TaskState -{ - Started = 0, - Finished, - Unknown -}; - -/// Used to mark status of shard partition tasks -struct TaskStateWithOwner -{ - TaskStateWithOwner() = default; - TaskStateWithOwner(TaskState state_, const String & owner_) : state(state_), owner(owner_) {} - - TaskState state{TaskState::Unknown}; - String owner; - - static String getData(TaskState state, const String & owner) - { - return TaskStateWithOwner(state, owner).toString(); - } - - String toString() - { - WriteBufferFromOwnString wb; - wb << static_cast(state) << "\n" << escape << owner; - return wb.str(); - } - - static TaskStateWithOwner fromString(const String & data) - { - ReadBufferFromString rb(data); - TaskStateWithOwner res; - UInt32 state; - - rb >> state >> "\n" >> escape >> res.owner; - - if (state >= static_cast(TaskState::Unknown)) - throw Exception("Unknown state " + data, ErrorCodes::LOGICAL_ERROR); - - res.state = static_cast(state); - return res; - } -}; - - -/// Hierarchical description of the tasks -struct ShardPartition; -struct TaskShard; -struct TaskTable; -struct TaskCluster; -struct ClusterPartition; - -using TasksPartition = std::map>; -using ShardInfo = Cluster::ShardInfo; -using TaskShardPtr = std::shared_ptr; -using TasksShard = std::vector; -using TasksTable = std::list; -using ClusterPartitions = std::map>; - - -/// Just destination partition of a shard -struct ShardPartition -{ - ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {} - - String getPartitionPath() const; - String getPartitionCleanStartPath() const; - String getCommonPartitionIsDirtyPath() const; - String getCommonPartitionIsCleanedPath() const; - String getPartitionActiveWorkersPath() const; - String getActiveWorkerPath() const; - String getPartitionShardsPath() const; - String getShardStatusPath() const; - - TaskShard & task_shard; - String name; -}; - - -struct ShardPriority -{ - UInt8 is_remote = 1; - size_t hostname_difference = 0; - UInt8 random = 0; - - static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) - { - return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random) - < std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random); - } -}; - - -struct TaskShard -{ - TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) {} - - TaskTable & task_table; - - ShardInfo info; - UInt32 numberInCluster() const { return info.shard_num; } - UInt32 indexInCluster() const { return info.shard_num - 1; } - - String getDescription() const; - String getHostNameExample() const; - - /// Used to sort clusters by their proximity - ShardPriority priority; - - /// Column with unique destination partitions (computed from engine_push_partition_key expr.) in the shard - ColumnWithTypeAndName partition_key_column; - - /// There is a task for each destination partition - TasksPartition partition_tasks; - - /// Which partitions have been checked for existence - /// If some partition from this lists is exists, it is in partition_tasks - std::set checked_partitions; - - /// Last CREATE TABLE query of the table of the shard - ASTPtr current_pull_table_create_query; - - /// Internal distributed tables - DatabaseAndTableName table_read_shard; - DatabaseAndTableName table_split_shard; -}; - - -/// Contains info about all shards that contain a partition -struct ClusterPartition -{ - double elapsed_time_seconds = 0; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - UInt64 blocks_copied = 0; - - UInt64 total_tries = 0; -}; - - -struct TaskTable -{ - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - String getPartitionPath(const String & partition_name) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /// A Distributed table definition used to split data - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /// Prioritized list of shards - TasksShard all_shards; - TasksShard local_shards; - - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String & partition_name) - { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine && random_engine); -}; - - -struct TaskCluster -{ - TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) - : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} - - void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Set (or update) settings and max_workers param - void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Base node for all tasks. Its structure: - /// workers/ - directory with active workers (amount of them is less or equal max_workers) - /// description - node with task configuration - /// table_table1/ - directories with per-partition copying status - String task_zookeeper_path; - - /// Database used to create temporary Distributed tables - String default_local_database; - - /// Limits number of simultaneous workers - UInt64 max_workers = 0; - - /// Base settings for pull and push - Settings settings_common; - /// Settings used to fetch data - Settings settings_pull; - /// Settings used to insert data - Settings settings_push; - - String clusters_prefix; - - /// Subtasks - TasksTable table_tasks; - - std::random_device random_device; - pcg64 random_engine; -}; - - -struct MultiTransactionInfo -{ - int32_t code; - Coordination::Requests requests; - Coordination::Responses responses; -}; - -// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) -std::shared_ptr createASTStorageDistributed( - const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) -{ - auto args = std::make_shared(); - args->children.emplace_back(std::make_shared(cluster_name)); - args->children.emplace_back(std::make_shared(database)); - args->children.emplace_back(std::make_shared(table)); - if (sharding_key_ast) - args->children.emplace_back(sharding_key_ast); - - auto engine = std::make_shared(); - engine->name = "Distributed"; - engine->arguments = args; - - auto storage = std::make_shared(); - storage->set(storage->engine, engine); - - return storage; -} - - -BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) -{ - return std::make_shared( - stream, - std::numeric_limits::max(), - std::numeric_limits::max()); -} - -Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) -{ - return squashStreamIntoOneBlock(stream)->read(); -} - - -/// Path getters - -String TaskTable::getPartitionPath(const String & partition_name) const -{ - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String ShardPartition::getPartitionCleanStartPath() const -{ - return getPartitionPath() + "/clean_start"; -} - -String ShardPartition::getPartitionPath() const -{ - return task_shard.task_table.getPartitionPath(name); -} - -String ShardPartition::getShardStatusPath() const -{ - // schema: //tables///shards/ - // e.g. /root/table_test.hits/201701/shards/1 - return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getPartitionShardsPath() const -{ - return getPartitionPath() + "/shards"; -} - -String ShardPartition::getPartitionActiveWorkersPath() const -{ - return getPartitionPath() + "/partition_active_workers"; -} - -String ShardPartition::getActiveWorkerPath() const -{ - return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getCommonPartitionIsDirtyPath() const -{ - return getPartitionPath() + "/is_dirty"; -} - -String ShardPartition::getCommonPartitionIsCleanedPath() const -{ - return getCommonPartitionIsDirtyPath() + "/cleaned"; -} - -String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const -{ - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/shards"; -} - -String DB::TaskShard::getDescription() const -{ - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String DB::TaskShard::getHostNameExample() const -{ - auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - - -static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) -{ - const auto & storage = storage_ast->as(); - return storage.partition_by || storage.order_by || storage.sample_by; -} - -static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) -{ - String storage_str = queryToString(storage_ast); - - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", - ErrorCodes::BAD_ARGUMENTS); - } - - if (isExtendedDefinitionStorage(storage_ast)) - { - if (storage.partition_by) - return storage.partition_by->clone(); - - static const char * all = "all"; - return std::make_shared(Field(all, strlen(all))); - } - else - { - bool is_replicated = startsWith(engine.name, "Replicated"); - size_t min_args = is_replicated ? 3 : 1; - - if (!engine.arguments) - throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); - - ASTPtr arguments_ast = engine.arguments->clone(); - ASTs & arguments = arguments_ast->children; - - if (arguments.size() < min_args) - throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); - - ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1]; - return makeASTFunction("toYYYYMM", month_arg->clone()); - } -} - - -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) -: task_cluster(parent) -{ - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) - { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) - { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { - /// Parse sequence of ... - for (const String & key : keys) - { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - - -static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) -{ - ShardPriority res; - - if (replicas.empty()) - return res; - - res.is_remote = 1; - for (auto & replica : replicas) - { - if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) - { - res.is_remote = 0; - break; - } - } - - res.hostname_difference = std::numeric_limits::max(); - for (auto & replica : replicas) - { - size_t difference = getHostNameDifference(local_hostname, replica.host_name); - res.hostname_difference = std::min(difference, res.hostname_difference); - } - - res.random = random; - return res; -} - -template -void TaskTable::initShards(RandomEngine && random_engine) -{ - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto & shard_info : cluster_pull->getShardsInfo()) - { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) - { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [] (const TaskShardPtr & lhs, UInt8 is_remote) - { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -void DB::TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - clusters_prefix = prefix + "remote_servers"; - if (!config.has(clusters_prefix)) - throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); - - Poco::Util::AbstractConfiguration::Keys tables_keys; - config.keys(prefix + "tables", tables_keys); - - for (const auto & table_key : tables_keys) - { - table_tasks.emplace_back(*this, config, prefix + "tables", table_key); - } -} - -void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - max_workers = config.getUInt64(prefix + "max_workers"); - - settings_common = Settings(); - if (config.has(prefix + "settings")) - settings_common.loadSettingsFromConfig(prefix + "settings", config); - - settings_pull = settings_common; - if (config.has(prefix + "settings_pull")) - settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); - - settings_push = settings_common; - if (config.has(prefix + "settings_push")) - settings_push.loadSettingsFromConfig(prefix + "settings_push", config); - - auto set_default_value = [] (auto && setting, auto && default_value) - { - setting = setting.changed ? setting.value : default_value; - }; - - /// Override important settings - settings_pull.readonly = 1; - settings_push.insert_distributed_sync = 1; - set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); - set_default_value(settings_pull.max_threads, 1); - set_default_value(settings_pull.max_block_size, 8192UL); - set_default_value(settings_pull.preferred_block_size_bytes, 0); - set_default_value(settings_push.insert_distributed_timeout, 0); -} - - -} // end of an anonymous namespace - class ClusterCopier { public: - ClusterCopier(const String & task_path_, - const String & host_id_, - const String & proxy_database_name_, + ClusterCopier(String task_path_, + String host_id_, + String proxy_database_name_, Context & context_) : - task_zookeeper_path(task_path_), - host_id(host_id_), - working_database_name(proxy_database_name_), + task_zookeeper_path(std::move(task_path_)), + host_id(std::move(host_id_)), + working_database_name(std::move(proxy_database_name_)), context(context_), log(&Poco::Logger::get("ClusterCopier")) { @@ -940,14 +221,14 @@ public: task_description_watch_zookeeper = zookeeper; String task_config_str; - Coordination::Stat stat; + Coordination::Stat stat{}; int code; zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); if (code) throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - LOG_DEBUG(log, "Loading description, zxid=" << task_descprtion_current_stat.czxid); + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); auto config = getConfigurationFromXMLString(task_config_str); /// Setup settings @@ -955,7 +236,7 @@ public: context.getSettingsRef() = task_cluster->settings_common; task_cluster_current_config = config; - task_descprtion_current_stat = stat; + task_description_current_stat = stat; } void updateConfigIfNeeded() @@ -1092,7 +373,7 @@ protected: { updateConfigIfNeeded(); - Coordination::Stat stat; + Coordination::Stat stat{}; zookeeper->get(workers_version_path, &stat); auto version = stat.version; zookeeper->get(workers_path, &stat); @@ -1186,7 +467,7 @@ protected: task_table.getPartitionIsDirtyPath(partition_name), task_table.getPartitionIsCleanedPath(partition_name) ); - Coordination::Stat stat; + Coordination::Stat stat{}; LogicalClock task_start_clock; if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) task_start_clock = LogicalClock(stat.mzxid); @@ -1264,7 +545,7 @@ protected: } /// Replaces ENGINE and table name in a create query - std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -1291,7 +572,7 @@ protected: public: UInt32 value; - WrappingUInt32(UInt32 _value) + explicit WrappingUInt32(UInt32 _value) : value(_value) {} @@ -1315,13 +596,20 @@ protected: /** Conforming Zxid definition. * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions + * + * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html + * + * Actually here is the definition of Zxid. + * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). + * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid + * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. */ class Zxid { public: WrappingUInt32 epoch; WrappingUInt32 counter; - Zxid(UInt64 _zxid) + explicit Zxid(UInt64 _zxid) : epoch(_zxid >> 32) , counter(_zxid) {} @@ -1338,6 +626,51 @@ protected: } }; + /* When multiple ClusterCopiers discover that the target partition is not empty, + * they will attempt to clean up this partition before proceeding to copying. + * + * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established + * based on a happens-before relation between the events. + * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. + * The fact of the partition hygiene is encoded by CleanStateClock. + * + * For you to know what mzxid means: + * + * ZooKeeper Stat Structure: + * The Stat structure for each znode in ZooKeeper is made up of the following fields: + * + * -- czxid + * The zxid of the change that caused this znode to be created. + * + * -- mzxid + * The zxid of the change that last modified this znode. + * + * -- ctime + * The time in milliseconds from epoch when this znode was created. + * + * -- mtime + * The time in milliseconds from epoch when this znode was last modified. + * + * -- version + * The number of changes to the data of this znode. + * + * -- cversion + * The number of changes to the children of this znode. + * + * -- aversion + * The number of changes to the ACL of this znode. + * + * -- ephemeralOwner + * The session id of the owner of this znode if the znode is an ephemeral node. + * If it is not an ephemeral node, it will be zero. + * + * -- dataLength + * The length of the data field of this znode. + * + * -- numChildren + * The number of children of this znode. + * */ + class LogicalClock { public: @@ -1345,7 +678,7 @@ protected: LogicalClock() = default; - LogicalClock(UInt64 _zxid) + explicit LogicalClock(UInt64 _zxid) : zxid(_zxid) {} @@ -1354,7 +687,7 @@ protected: return bool(zxid); } - // happens-before relation with a reasonable time bound + /// happens-before relation with a reasonable time bound bool happensBefore(const LogicalClock & other) const { return !zxid @@ -1366,13 +699,14 @@ protected: return happensBefore(other); } - // strict equality check + /// strict equality check bool operator==(const LogicalClock & other) const { return zxid == other.zxid; } }; + class CleanStateClock { public: @@ -1404,7 +738,7 @@ protected: const String & clean_state_path) : stale(std::make_shared(false)) { - Coordination::Stat stat; + Coordination::Stat stat{}; String _some_data; auto watch_callback = [stale = stale] (const Coordination::WatchResponse & rsp) @@ -1462,13 +796,13 @@ protected: const String current_shards_path = task_partition.getPartitionShardsPath(); const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirt_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id); + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); } catch (const Coordination::Exception & e) { @@ -1482,7 +816,7 @@ protected: throw; } - Coordination::Stat stat; + Coordination::Stat stat{}; if (zookeeper->exists(current_partition_active_workers_dir, &stat)) { if (stat.numChildren != 0) @@ -1517,7 +851,7 @@ protected: // Lock the dirty flag zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); /// Remove all status nodes { @@ -1556,9 +890,9 @@ protected: { zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); if (my_clock.clean_state_version) - zookeeper->set(is_dirt_cleaned_path, host_id, my_clock.clean_state_version.value()); + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); else - zookeeper->create(is_dirt_cleaned_path, host_id, zkutil::CreateMode::Persistent); + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); } else { @@ -1582,7 +916,7 @@ protected: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) { - /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint bool previous_shard_is_instantly_finished = false; /// Process each partition that is present in cluster @@ -1594,6 +928,7 @@ protected: ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. TasksShard expected_shards; UInt64 num_failed_shards = 0; @@ -1637,6 +972,8 @@ protected: } auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. if (it_shard_partition == shard->partition_tasks.end()) throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); auto & partition = it_shard_partition->second; @@ -1738,6 +1075,7 @@ protected: Error, }; + /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { PartitionTaskStatus res; @@ -1770,6 +1108,8 @@ protected: TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + const size_t number_of_splits = task_table.number_of_splits; + UNUSED(number_of_splits); /// We need to update table definitions for each partition, it could be changed after ALTER createShardInternalTables(timeouts, task_shard); @@ -1777,7 +1117,7 @@ protected: auto zookeeper = context.getZooKeeper(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); const String current_task_is_active_path = task_partition.getActiveWorkerPath(); const String current_task_status_path = task_partition.getShardStatusPath(); @@ -1803,12 +1143,15 @@ protected: }; /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "") + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false, size_t current_piece_number = 0) { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) + query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; if (!limit.empty()) @@ -1823,11 +1166,11 @@ protected: LOG_DEBUG(log, "Processing " << current_task_status_path); - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); LogicalClock task_start_clock; { - Coordination::Stat stat; + Coordination::Stat stat{}; if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) task_start_clock = LogicalClock(stat.mzxid); } @@ -1887,7 +1230,8 @@ protected: } // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled."); + LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; @@ -1900,7 +1244,8 @@ protected: if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") { zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", *zookeeper, host_id); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); // Maybe we are the first worker ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); UInt64 count; @@ -1916,7 +1261,7 @@ protected: if (count != 0) { - Coordination::Stat stat_shards; + Coordination::Stat stat_shards{}; zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning @@ -1938,7 +1283,7 @@ protected: /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1955,15 +1300,17 @@ protected: /// Try create table (if not exists) on each shard { - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, create_query_push_ast, &task_cluster->settings_push, + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << " have been created on " << shards - << " shards of " << task_table.cluster_push->getShardCount()); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } /// Do the copying @@ -2073,7 +1420,7 @@ protected: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -2265,7 +1612,7 @@ protected: UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); num_successful_executions = 0; - auto increment_and_check_exit = [&] () + auto increment_and_check_exit = [&] () -> bool { ++num_successful_executions; return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; @@ -2348,7 +1695,7 @@ private: ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_descprtion_current_stat{}; + Coordination::Stat task_description_current_stat{}; std::unique_ptr task_cluster; @@ -2373,12 +1720,11 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) config_xml_path = config().getString("config-file"); task_path = config().getString("task-path"); - log_level = config().getString("log-level", "debug"); + log_level = config().getString("log-level", "trace"); is_safe_mode = config().has("safe-mode"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); - // process_id is '#_' time_t timestamp = Poco::Timestamp().epochTime(); auto curr_pid = Poco::Process::id(); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 89f45df8686..fe228fd6194 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -71,11 +71,9 @@ private: void mainImpl(); - void setupLogging(); - std::string config_xml_path; std::string task_path; - std::string log_level = "debug"; + std::string log_level = "trace"; bool is_safe_mode = false; double copy_fault_probability = 0; bool is_help = false; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index a25ae7b973c..5f14604fbf9 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -1,8 +1,746 @@ -// -// Created by jakalletti on 2/7/20. -// +#pragma once -#ifndef CLICKHOUSE_INTERNALS_H -#define CLICKHOUSE_INTERNALS_H +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NO_ZOOKEEPER; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; + extern const int UNFINISHED; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + +namespace +{ + +using DatabaseAndTableName = std::pair; + +String getQuotedTable(const String & database, const String & table) +{ + if (database.empty()) + { + return backQuoteIfNeed(table); + } + + return backQuoteIfNeed(database) + "." + backQuoteIfNeed(table); +} + +String getQuotedTable(const DatabaseAndTableName & db_and_table) +{ + return getQuotedTable(db_and_table.first, db_and_table.second); +} + + +enum class TaskState +{ + Started = 0, + Finished, + Unknown +}; + +/// Used to mark status of shard partition tasks +struct TaskStateWithOwner +{ + TaskStateWithOwner() = default; + TaskStateWithOwner(TaskState state_, const String & owner_) : state(state_), owner(owner_) {} + + TaskState state{TaskState::Unknown}; + String owner; + + static String getData(TaskState state, const String & owner) + { + return TaskStateWithOwner(state, owner).toString(); + } + + String toString() + { + WriteBufferFromOwnString wb; + wb << static_cast(state) << "\n" << escape << owner; + return wb.str(); + } + + static TaskStateWithOwner fromString(const String & data) + { + ReadBufferFromString rb(data); + TaskStateWithOwner res; + UInt32 state; + + rb >> state >> "\n" >> escape >> res.owner; + + if (state >= static_cast(TaskState::Unknown)) + throw Exception("Unknown state " + data, ErrorCodes::LOGICAL_ERROR); + + res.state = static_cast(state); + return res; + } +}; + + +/// Hierarchical description of the tasks +struct ShardPartition; +struct TaskShard; +struct TaskTable; +struct TaskCluster; +struct ClusterPartition; + +using TasksPartition = std::map>; +using ShardInfo = Cluster::ShardInfo; +using TaskShardPtr = std::shared_ptr; +using TasksShard = std::vector; +using TasksTable = std::list; +using ClusterPartitions = std::map>; + + +/// Just destination partition of a shard +/// I don't know what this comment means. +/// In short, when we discovered what shards contain currently processing partition, +/// This class describes a partition (name) that is stored on the shard (parent). +struct ShardPartition +{ + ShardPartition(TaskShard & parent, String name_quoted_) : task_shard(parent), name(std::move(name_quoted_)) {} + + String getPartitionPath() const; + String getPartitionCleanStartPath() const; + String getCommonPartitionIsDirtyPath() const; + String getCommonPartitionIsCleanedPath() const; + String getPartitionActiveWorkersPath() const; + String getActiveWorkerPath() const; + String getPartitionShardsPath() const; + String getShardStatusPath() const; + + TaskShard & task_shard; + String name; +}; + + +struct ShardPriority +{ + UInt8 is_remote = 1; + size_t hostname_difference = 0; + UInt8 random = 0; + + static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) + { + return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random) + < std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random); + } +}; + +/// Tables has many shards and table's partiton can be stored on different shards. +/// When we copy partition we have to discover it's shards (shards which store this partition) +/// For simplier retrieval of which partitions are stored in particular shard we created TaskShard. +struct TaskShard +{ + TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) {} + + TaskTable & task_table; + + ShardInfo info; + UInt32 numberInCluster() const { return info.shard_num; } + UInt32 indexInCluster() const { return info.shard_num - 1; } + + String getDescription() const; + String getHostNameExample() const; + + /// Used to sort clusters by their proximity + ShardPriority priority; + + /// Column with unique destination partitions (computed from engine_push_partition_key expr.) in the shard + ColumnWithTypeAndName partition_key_column; + + /// There is a task for each destination partition + TasksPartition partition_tasks; + + /// Which partitions have been checked for existence + /// If some partition from this lists is exists, it is in partition_tasks + std::set checked_partitions; + + /// Last CREATE TABLE query of the table of the shard + ASTPtr current_pull_table_create_query; + + /// Internal distributed tables + DatabaseAndTableName table_read_shard; + DatabaseAndTableName table_split_shard; +}; + + +/// Contains info about all shards that contain a partition +struct ClusterPartition +{ + double elapsed_time_seconds = 0; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + UInt64 blocks_copied = 0; + + UInt64 total_tries = 0; +}; + + +struct TaskTable +{ + TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, + const String & table_key); + + TaskCluster & task_cluster; + + String getPartitionPath(const String & partition_name) const; + [[maybe_unused]] String getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const; + String getPartitionIsDirtyPath(const String & partition_name) const; + String getPartitionIsCleanedPath(const String & partition_name) const; + String getPartitionTaskStatusPath(const String & partition_name) const; + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; + + String name_in_config; + + /// Used as task ID + String table_id; + + /// Source cluster and table + String cluster_pull_name; + DatabaseAndTableName table_pull; + + /// Destination cluster and table + String cluster_push_name; + DatabaseAndTableName table_push; + + /// Storage of destination table + String engine_push_str; + ASTPtr engine_push_ast; + ASTPtr engine_push_partition_key_ast; + + /// A Distributed table definition used to split data + String sharding_key_str; + ASTPtr sharding_key_ast; + ASTPtr engine_split_ast; + + /// Additional WHERE expression to filter input data + String where_condition_str; + ASTPtr where_condition_ast; + + /// Resolved clusters + ClusterPtr cluster_pull; + ClusterPtr cluster_push; + + /// Filter partitions that should be copied + bool has_enabled_partitions = false; + Strings enabled_partitions; + NameSet enabled_partitions_set; + + /// Prioritized list of shards + /// all_shards contains information about all shards in the table. + /// So we have to check whether particular shard have current partiton or not while processing. + TasksShard all_shards; + TasksShard local_shards; + + /// All partitions of the current table. + ClusterPartitions cluster_partitions; + NameSet finished_cluster_partitions; + + /// Parition names to process in user-specified order + Strings ordered_partition_names; + + ClusterPartition & getClusterPartition(const String & partition_name) + { + auto it = cluster_partitions.find(partition_name); + if (it == cluster_partitions.end()) + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); + return it->second; + } + + Stopwatch watch; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + + template + void initShards(RandomEngine && random_engine); +}; + + +struct TaskCluster +{ + TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) + : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} + + void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); + + /// Set (or update) settings and max_workers param + void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); + + /// Base node for all tasks. Its structure: + /// workers/ - directory with active workers (amount of them is less or equal max_workers) + /// description - node with task configuration + /// table_table1/ - directories with per-partition copying status + String task_zookeeper_path; + + /// Database used to create temporary Distributed tables + String default_local_database; + + /// Limits number of simultaneous workers + UInt64 max_workers = 0; + + /// Base settings for pull and push + Settings settings_common; + /// Settings used to fetch data + Settings settings_pull; + /// Settings used to insert data + Settings settings_push; + + String clusters_prefix; + + /// Subtasks + TasksTable table_tasks; + + std::random_device random_device; + pcg64 random_engine; +}; + + + +struct MultiTransactionInfo +{ + int32_t code; + Coordination::Requests requests; + Coordination::Responses responses; +}; + +// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) +std::shared_ptr createASTStorageDistributed( + const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) +{ + auto args = std::make_shared(); + args->children.emplace_back(std::make_shared(cluster_name)); + args->children.emplace_back(std::make_shared(database)); + args->children.emplace_back(std::make_shared(table)); + if (sharding_key_ast) + args->children.emplace_back(sharding_key_ast); + + auto engine = std::make_shared(); + engine->name = "Distributed"; + engine->arguments = args; + + auto storage = std::make_shared(); + storage->set(storage->engine, engine); + + return storage; +} + + +BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) +{ + return std::make_shared( + stream, + std::numeric_limits::max(), + std::numeric_limits::max()); +} + +Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) +{ + return squashStreamIntoOneBlock(stream)->read(); +} + + +/// Path getters + +String TaskTable::getPartitionPath(const String & partition_name) const +{ + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +String TaskTable::getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const +{ + return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits +} + +String ShardPartition::getPartitionCleanStartPath() const +{ + return getPartitionPath() + "/clean_start"; +} + +String ShardPartition::getPartitionPath() const +{ + return task_shard.task_table.getPartitionPath(name); +} + +String ShardPartition::getShardStatusPath() const +{ + // schema: //tables/
//shards/ + // e.g. /root/table_test.hits/201701/shards/1 + return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getPartitionShardsPath() const +{ + return getPartitionPath() + "/shards"; +} + +String ShardPartition::getPartitionActiveWorkersPath() const +{ + return getPartitionPath() + "/partition_active_workers"; +} + +String ShardPartition::getActiveWorkerPath() const +{ + return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getCommonPartitionIsDirtyPath() const +{ + return getPartitionPath() + "/is_dirty"; +} + +String ShardPartition::getCommonPartitionIsCleanedPath() const +{ + return getCommonPartitionIsDirtyPath() + "/cleaned"; +} + +String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/is_dirty"; +} + +String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const +{ + return getPartitionIsDirtyPath(partition_name) + "/cleaned"; +} + +String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/shards"; +} + +String TaskShard::getDescription() const +{ + std::stringstream ss; + ss << "N" << numberInCluster() + << " (having a replica " << getHostNameExample() + << ", pull table " + getQuotedTable(task_table.table_pull) + << " of cluster " + task_table.cluster_pull_name << ")"; + return ss.str(); +} + +String TaskShard::getHostNameExample() const +{ + auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + return replicas.at(0).readableString(); +} + + +static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) +{ + const auto & storage = storage_ast->as(); + return storage.partition_by || storage.order_by || storage.sample_by; +} + +static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (isExtendedDefinitionStorage(storage_ast)) + { + if (storage.partition_by) + return storage.partition_by->clone(); + + static const char * all = "all"; + return std::make_shared(Field(all, strlen(all))); + } + else + { + bool is_replicated = startsWith(engine.name, "Replicated"); + size_t min_args = is_replicated ? 3 : 1; + + if (!engine.arguments) + throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); + + ASTPtr arguments_ast = engine.arguments->clone(); + ASTs & arguments = arguments_ast->children; + + if (arguments.size() < min_args) + throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); + + ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1]; + return makeASTFunction("toYYYYMM", month_arg->clone()); + } +} + + +TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, + const String & table_key) + : task_cluster(parent) +{ + String table_prefix = prefix_ + "." + table_key + "."; + + name_in_config = table_key; + + number_of_splits = config.getUInt64("number_of_splits", 10); + + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); + cluster_push_name = config.getString(table_prefix + "cluster_push"); + + table_pull.first = config.getString(table_prefix + "database_pull"); + table_pull.second = config.getString(table_prefix + "table_pull"); + + table_push.first = config.getString(table_prefix + "database_push"); + table_push.second = config.getString(table_prefix + "table_push"); + + /// Used as node name in ZooKeeper + table_id = escapeForFileName(cluster_push_name) + + "." + escapeForFileName(table_push.first) + + "." + escapeForFileName(table_push.second); + + engine_push_str = config.getString(table_prefix + "engine"); + { + ParserStorage parser_storage; + engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + } + + sharding_key_str = config.getString(table_prefix + "sharding_key"); + { + ParserExpressionWithOptionalAlias parser_expression(false); + sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); + engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + } + + where_condition_str = config.getString(table_prefix + "where_condition", ""); + if (!where_condition_str.empty()) + { + ParserExpressionWithOptionalAlias parser_expression(false); + where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); + + // Will use canonical expression form + where_condition_str = queryToString(where_condition_ast); + } + + String enabled_partitions_prefix = table_prefix + "enabled_partitions"; + has_enabled_partitions = config.has(enabled_partitions_prefix); + + if (has_enabled_partitions) + { + Strings keys; + config.keys(enabled_partitions_prefix, keys); + + if (keys.empty()) + { + /// Parse list of partition from space-separated string + String partitions_str = config.getString(table_prefix + "enabled_partitions"); + boost::trim_if(partitions_str, isWhitespaceASCII); + boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); + } + else + { + /// Parse sequence of ... + for (const String & key : keys) + { + if (!startsWith(key, "partition")) + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); + } + } + + std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + } +} + + +static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) +{ + ShardPriority res; + + if (replicas.empty()) + return res; + + res.is_remote = 1; + for (auto & replica : replicas) + { + if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) + { + res.is_remote = 0; + break; + } + } + + res.hostname_difference = std::numeric_limits::max(); + for (auto & replica : replicas) + { + size_t difference = getHostNameDifference(local_hostname, replica.host_name); + res.hostname_difference = std::min(difference, res.hostname_difference); + } + + res.random = random; + return res; +} + +template +void TaskTable::initShards(RandomEngine && random_engine) +{ + const String & fqdn_name = getFQDNOrHostName(); + std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); + + // Compute the priority + for (auto & shard_info : cluster_pull->getShardsInfo()) + { + TaskShardPtr task_shard = std::make_shared(*this, shard_info); + const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); + task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); + + all_shards.emplace_back(task_shard); + } + + // Sort by priority + std::sort(all_shards.begin(), all_shards.end(), + [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) + { + return ShardPriority::greaterPriority(lhs->priority, rhs->priority); + }); + + // Cut local shards + auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, + [] (const TaskShardPtr & lhs, UInt8 is_remote) + { + return lhs->priority.is_remote < is_remote; + }); + + local_shards.assign(all_shards.begin(), it_first_remote); +} + + +void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) +{ + String prefix = base_key.empty() ? "" : base_key + "."; + + clusters_prefix = prefix + "remote_servers"; + if (!config.has(clusters_prefix)) + throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); + + Poco::Util::AbstractConfiguration::Keys tables_keys; + config.keys(prefix + "tables", tables_keys); + + for (const auto & table_key : tables_keys) + { + table_tasks.emplace_back(*this, config, prefix + "tables", table_key); + } +} + +void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) +{ + String prefix = base_key.empty() ? "" : base_key + "."; + + max_workers = config.getUInt64(prefix + "max_workers"); + + settings_common = Settings(); + if (config.has(prefix + "settings")) + settings_common.loadSettingsFromConfig(prefix + "settings", config); + + settings_pull = settings_common; + if (config.has(prefix + "settings_pull")) + settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); + + settings_push = settings_common; + if (config.has(prefix + "settings_push")) + settings_push.loadSettingsFromConfig(prefix + "settings_push", config); + + auto set_default_value = [] (auto && setting, auto && default_value) + { + setting = setting.changed ? setting.value : default_value; + }; + + /// Override important settings + settings_pull.readonly = 1; + settings_push.insert_distributed_sync = 1; + set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); + set_default_value(settings_pull.max_threads, 1); + set_default_value(settings_pull.max_block_size, 8192UL); + set_default_value(settings_pull.preferred_block_size_bytes, 0); + set_default_value(settings_push.insert_distributed_timeout, 0); +} + + +} // end of an anonymous namespace +} -#endif //CLICKHOUSE_INTERNALS_H From 22789a3b55ab392cf3de32db082224dbea0a72b6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 11 Feb 2020 21:34:48 +0300 Subject: [PATCH 003/147] ShardPartitionPiece task added --- dbms/programs/copier/CMakeLists.txt | 2 +- dbms/programs/copier/ClusterCopier.cpp | 1879 --------------------- dbms/programs/copier/ClusterCopier.h | 1570 ++++++++++++++++- dbms/programs/copier/ClusterCopierApp.cpp | 173 ++ dbms/programs/copier/ClusterCopierApp.h | 87 + dbms/programs/copier/Internals.h | 74 +- dbms/programs/copier/ZookeeperStaff.h | 224 +++ 7 files changed, 2052 insertions(+), 1957 deletions(-) delete mode 100644 dbms/programs/copier/ClusterCopier.cpp create mode 100644 dbms/programs/copier/ClusterCopierApp.cpp create mode 100644 dbms/programs/copier/ClusterCopierApp.h create mode 100644 dbms/programs/copier/ZookeeperStaff.h diff --git a/dbms/programs/copier/CMakeLists.txt b/dbms/programs/copier/CMakeLists.txt index 8e13040b29d..9852dc92242 100644 --- a/dbms/programs/copier/CMakeLists.txt +++ b/dbms/programs/copier/CMakeLists.txt @@ -1,4 +1,4 @@ -set(CLICKHOUSE_COPIER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/ClusterCopier.cpp) +set(CLICKHOUSE_COPIER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/ClusterCopierApp.cpp) set(CLICKHOUSE_COPIER_LINK PRIVATE clickhouse_common_zookeeper clickhouse_parsers clickhouse_functions clickhouse_table_functions clickhouse_aggregate_functions clickhouse_dictionaries string_utils ${Poco_XML_LIBRARY} PUBLIC daemon) set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp deleted file mode 100644 index e8b3018a861..00000000000 --- a/dbms/programs/copier/ClusterCopier.cpp +++ /dev/null @@ -1,1879 +0,0 @@ -#include "ClusterCopier.h" - -#include "Internals.h" - -namespace DB -{ - - -using ConfigurationPtr = Poco::AutoPtr; - -static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) -{ - std::stringstream ss(xml_data); - Poco::XML::InputSource input_source{ss}; - return {new Poco::Util::XMLConfiguration{&input_source}}; -} - - -class ClusterCopier -{ -public: - - ClusterCopier(String task_path_, - String host_id_, - String proxy_database_name_, - Context & context_) - : - task_zookeeper_path(std::move(task_path_)), - host_id(std::move(host_id_)), - working_database_name(std::move(proxy_database_name_)), - context(context_), - log(&Poco::Logger::get("ClusterCopier")) - { - } - - void init() - { - auto zookeeper = context.getZooKeeper(); - - task_description_watch_callback = [this] (const Coordination::WatchResponse & response) - { - if (response.error != Coordination::ZOK) - return; - UInt64 version = ++task_descprtion_version; - LOG_DEBUG(log, "Task description should be updated, local version " << version); - }; - - task_description_path = task_zookeeper_path + "/description"; - task_cluster = std::make_unique(task_zookeeper_path, working_database_name); - - reloadTaskDescription(); - task_cluster_initial_config = task_cluster_current_config; - - task_cluster->loadTasks(*task_cluster_initial_config); - context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); - - /// Set up shards and their priority - task_cluster->random_engine.seed(task_cluster->random_device()); - for (auto & task_table : task_cluster->table_tasks) - { - task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - task_table.cluster_push = context.getCluster(task_table.cluster_push_name); - task_table.initShards(task_cluster->random_engine); - } - - LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); - - /// Do not initialize tables, will make deferred initialization in process() - - zookeeper->createAncestors(getWorkersPathVersion() + "/"); - zookeeper->createAncestors(getWorkersPath() + "/"); - } - - template - decltype(auto) retry(T && func, UInt64 max_tries = 100) - { - std::exception_ptr exception; - - for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) - { - try - { - return func(); - } - catch (...) - { - exception = std::current_exception(); - if (try_number < max_tries) - { - tryLogCurrentException(log, "Will retry"); - std::this_thread::sleep_for(default_sleep_time); - } - } - } - - std::rethrow_exception(exception); - } - - - void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) - { - TaskTable & task_table = task_shard->task_table; - - LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); - - auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; - auto existing_partitions_names = retry(get_partitions, 60); - Strings filtered_partitions_names; - Strings missing_partitions; - - /// Check that user specified correct partition names - auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) - { - MutableColumnPtr column_dummy = type->createColumn(); - ReadBufferFromString rb(partition_text_quoted); - - try - { - type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); - } - catch (Exception & e) - { - throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); - } - }; - - if (task_table.has_enabled_partitions) - { - /// Process partition in order specified by - for (const String & partition_name : task_table.enabled_partitions) - { - /// Check that user specified correct partition names - check_partition_format(task_shard->partition_key_column.type, partition_name); - - auto it = existing_partitions_names.find(partition_name); - - /// Do not process partition if it is not in enabled_partitions list - if (it == existing_partitions_names.end()) - { - missing_partitions.emplace_back(partition_name); - continue; - } - - filtered_partitions_names.emplace_back(*it); - } - - for (const String & partition_name : existing_partitions_names) - { - if (!task_table.enabled_partitions_set.count(partition_name)) - { - LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " - << "enabled_partitions of " << task_table.table_id); - } - } - } - else - { - for (const String & partition_name : existing_partitions_names) - filtered_partitions_names.emplace_back(partition_name); - } - - for (const String & partition_name : filtered_partitions_names) - { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name)); - task_shard->checked_partitions.emplace(partition_name, true); - } - - if (!missing_partitions.empty()) - { - std::stringstream ss; - for (const String & missing_partition : missing_partitions) - ss << " " << missing_partition; - - LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " - << task_shard->getDescription() << " :" << ss.str()); - } - - LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); - } - - /// Compute set of partitions, assume set of partitions aren't changed during the processing - void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) - { - /// Fetch partitions list from a shard - { - ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); - - for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); - - LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); - thread_pool.wait(); - } - } - - void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) - { - auto local_task_description_path = task_path + "/description"; - - String task_config_str; - { - ReadBufferFromFile in(task_file); - readStringUntilEOF(task_config_str, in); - } - if (task_config_str.empty()) - return; - - auto zookeeper = context.getZooKeeper(); - - zookeeper->createAncestors(local_task_description_path); - auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - if (code && force) - zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - - LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); - } - - void reloadTaskDescription() - { - auto zookeeper = context.getZooKeeper(); - task_description_watch_zookeeper = zookeeper; - - String task_config_str; - Coordination::Stat stat{}; - int code; - - zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code) - throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - - LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); - auto config = getConfigurationFromXMLString(task_config_str); - - /// Setup settings - task_cluster->reloadSettings(*config); - context.getSettingsRef() = task_cluster->settings_common; - - task_cluster_current_config = config; - task_description_current_stat = stat; - } - - void updateConfigIfNeeded() - { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); - - if (!is_outdated_version && !is_expired_session) - return; - - LOG_DEBUG(log, "Updating task description"); - reloadTaskDescription(); - - task_descprtion_current_version = version_to_update; - } - - void process(const ConnectionTimeouts & timeouts) - { - for (TaskTable & task_table : task_cluster->table_tasks) - { - LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); - - if (task_table.all_shards.empty()) - continue; - - /// Discover partitions of each shard and total set of partitions - if (!task_table.has_enabled_partitions) - { - /// If there are no specified enabled_partitions, we must discover them manually - discoverTablePartitions(timeouts, task_table); - - /// After partitions of each shard are initialized, initialize cluster partitions - for (const TaskShardPtr & task_shard : task_table.all_shards) - { - for (const auto & partition_elem : task_shard->partition_tasks) - { - const String & partition_name = partition_elem.first; - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - } - } - - for (auto & partition_elem : task_table.cluster_partitions) - { - const String & partition_name = partition_elem.first; - - for (const TaskShardPtr & task_shard : task_table.all_shards) - task_shard->checked_partitions.emplace(partition_name); - - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - else - { - /// If enabled_partitions are specified, assume that each shard has all partitions - /// We will refine partition set of each shard in future - - for (const String & partition_name : task_table.enabled_partitions) - { - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - - task_table.watch.restart(); - - /// Retry table processing - bool table_is_done = false; - for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) - { - if (tryProcessTable(timeouts, task_table)) - { - table_is_done = true; - break; - } - } - - if (!table_is_done) - { - throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", - ErrorCodes::UNFINISHED); - } - } - } - - /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true) - { - is_safe_mode = is_safe_mode_; - } - - void setCopyFaultProbability(double copy_fault_probability_) - { - copy_fault_probability = copy_fault_probability_; - } - - -protected: - - String getWorkersPath() const - { - return task_cluster->task_zookeeper_path + "/task_active_workers"; - } - - String getWorkersPathVersion() const - { - return getWorkersPath() + "_version"; - } - - String getCurrentWorkerNodePath() const - { - return getWorkersPath() + "/" + host_id; - } - - zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( - const zkutil::ZooKeeperPtr & zookeeper, - const String & description, - bool unprioritized) - { - std::chrono::milliseconds current_sleep_time = default_sleep_time; - static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec - - if (unprioritized) - std::this_thread::sleep_for(current_sleep_time); - - String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); - - UInt64 num_bad_version_errors = 0; - - while (true) - { - updateConfigIfNeeded(); - - Coordination::Stat stat{}; - zookeeper->get(workers_version_path, &stat); - auto version = stat.version; - zookeeper->get(workers_path, &stat); - - if (static_cast(stat.numChildren) >= task_cluster->max_workers) - { - LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" - << ". Postpone processing " << description); - - if (unprioritized) - current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); - - std::this_thread::sleep_for(current_sleep_time); - num_bad_version_errors = 0; - } - else - { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); - ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); - Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); - - if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); - - if (code == Coordination::ZBADVERSION) - { - ++num_bad_version_errors; - - /// Try to make fast retries - if (num_bad_version_errors > 3) - { - LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); - std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); - std::this_thread::sleep_for(random_sleep_time); - num_bad_version_errors = 0; - } - } - else - throw Coordination::Exception(code); - } - } - } - - /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. - * State of some task could change during the processing. - * We have to ensure that all shards have the finished state and there is no dirty flag. - * Moreover, we have to check status twice and check zxid, because state can change during the checking. - */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); - - auto zookeeper = context.getZooKeeper(); - - Strings status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getPartitionIsDirtyPath(partition_name), - task_table.getPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - } - - /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) - { - const ASTs & column_asts = query_ast->as().columns_list->columns->children; - auto new_columns = std::make_shared(); - - for (const ASTPtr & column_ast : column_asts) - { - const auto & column = column_ast->as(); - - if (!column.default_specifier.empty()) - { - ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); - if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) - continue; - } - - new_columns->children.emplace_back(column_ast->clone()); - } - - ASTPtr new_query_ast = query_ast->clone(); - auto & new_query = new_query_ast->as(); - - auto new_columns_list = std::make_shared(); - new_columns_list->set(new_columns_list->columns, new_columns); - if (auto indices = query_ast->as()->columns_list->indices) - new_columns_list->set(new_columns_list->indices, indices->clone()); - - new_query.replace(new_query.columns_list, new_columns_list); - - return new_query_ast; - } - - /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) - { - const auto & create = create_query_ast->as(); - auto res = std::make_shared(create); - - if (create.storage == nullptr || new_storage_ast == nullptr) - throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); - - res->database = new_table.first; - res->table = new_table.second; - - res->children.clear(); - res->set(res->columns_list, create.columns_list->clone()); - res->set(res->storage, new_storage_ast->clone()); - - return res; - } - - /** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. - * We assume that we compare values that are not too far away. - * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. - */ - class WrappingUInt32 - { - public: - UInt32 value; - - explicit WrappingUInt32(UInt32 _value) - : value(_value) - {} - - bool operator<(const WrappingUInt32 & other) const - { - return value != other.value && *this <= other; - } - - bool operator<=(const WrappingUInt32 & other) const - { - const UInt32 HALF = 1 << 31; - return (value <= other.value && other.value - value < HALF) - || (value > other.value && value - other.value > HALF); - } - - bool operator==(const WrappingUInt32 & other) const - { - return value == other.value; - } - }; - - /** Conforming Zxid definition. - * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions - * - * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html - * - * Actually here is the definition of Zxid. - * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). - * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid - * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. - */ - class Zxid - { - public: - WrappingUInt32 epoch; - WrappingUInt32 counter; - explicit Zxid(UInt64 _zxid) - : epoch(_zxid >> 32) - , counter(_zxid) - {} - - bool operator<=(const Zxid & other) const - { - return (epoch < other.epoch) - || (epoch == other.epoch && counter <= other.counter); - } - - bool operator==(const Zxid & other) const - { - return epoch == other.epoch && counter == other.counter; - } - }; - - /* When multiple ClusterCopiers discover that the target partition is not empty, - * they will attempt to clean up this partition before proceeding to copying. - * - * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established - * based on a happens-before relation between the events. - * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. - * The fact of the partition hygiene is encoded by CleanStateClock. - * - * For you to know what mzxid means: - * - * ZooKeeper Stat Structure: - * The Stat structure for each znode in ZooKeeper is made up of the following fields: - * - * -- czxid - * The zxid of the change that caused this znode to be created. - * - * -- mzxid - * The zxid of the change that last modified this znode. - * - * -- ctime - * The time in milliseconds from epoch when this znode was created. - * - * -- mtime - * The time in milliseconds from epoch when this znode was last modified. - * - * -- version - * The number of changes to the data of this znode. - * - * -- cversion - * The number of changes to the children of this znode. - * - * -- aversion - * The number of changes to the ACL of this znode. - * - * -- ephemeralOwner - * The session id of the owner of this znode if the znode is an ephemeral node. - * If it is not an ephemeral node, it will be zero. - * - * -- dataLength - * The length of the data field of this znode. - * - * -- numChildren - * The number of children of this znode. - * */ - - class LogicalClock - { - public: - std::optional zxid; - - LogicalClock() = default; - - explicit LogicalClock(UInt64 _zxid) - : zxid(_zxid) - {} - - bool hasHappened() const - { - return bool(zxid); - } - - /// happens-before relation with a reasonable time bound - bool happensBefore(const LogicalClock & other) const - { - return !zxid - || (other.zxid && *zxid <= *other.zxid); - } - - bool operator<=(const LogicalClock & other) const - { - return happensBefore(other); - } - - /// strict equality check - bool operator==(const LogicalClock & other) const - { - return zxid == other.zxid; - } - }; - - - class CleanStateClock - { - public: - LogicalClock discovery_zxid; - std::optional discovery_version; - - LogicalClock clean_state_zxid; - std::optional clean_state_version; - - std::shared_ptr stale; - - bool is_clean() const - { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); - } - - bool is_stale() const - { - return stale->load(); - } - - CleanStateClock( - const zkutil::ZooKeeperPtr & zookeeper, - const String & discovery_path, - const String & clean_state_path) - : stale(std::make_shared(false)) - { - Coordination::Stat stat{}; - String _some_data; - auto watch_callback = - [stale = stale] (const Coordination::WatchResponse & rsp) - { - auto logger = &Poco::Logger::get("ClusterCopier"); - if (rsp.error == Coordination::ZOK) - { - switch (rsp.type) - { - case Coordination::CREATED: - LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); - stale->store(true); - break; - case Coordination::CHANGED: - LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); - stale->store(true); - } - } - }; - if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) - { - discovery_zxid = LogicalClock(stat.mzxid); - discovery_version = stat.version; - } - if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) - { - clean_state_zxid = LogicalClock(stat.mzxid); - clean_state_version = stat.version; - } - } - - bool operator==(const CleanStateClock & other) const - { - return !is_stale() - && !other.is_stale() - && discovery_zxid == other.discovery_zxid - && discovery_version == other.discovery_version - && clean_state_zxid == other.clean_state_zxid - && clean_state_version == other.clean_state_version; - } - - bool operator!=(const CleanStateClock & other) const - { - return !(*this == other); - } - }; - - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) - { - if (is_safe_mode) - throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); - - TaskTable & task_table = task_partition.task_shard.task_table; - - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - - zkutil::EphemeralNodeHolder::Ptr cleaner_holder; - try - { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - - throw; - } - - Coordination::Stat stat{}; - if (zookeeper->exists(current_partition_active_workers_dir, &stat)) - { - if (stat.numChildren != 0) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - else - { - zookeeper->remove(current_partition_active_workers_dir); - } - } - - { - zkutil::EphemeralNodeHolder::Ptr active_workers_lock; - try - { - active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); - return false; - } - - throw; - } - - // Lock the dirty flag - zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - /// Remove all status nodes - { - Strings children; - if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) - for (const auto & child : children) - { - zookeeper->removeRecursive(current_shards_path + "/" + child); - } - } - - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); - query += " DROP PARTITION " + task_partition.name + ""; - - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - - ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - /// It is important, DROP PARTITION must be done synchronously - settings_push.replication_alter_partitions_sync = 2; - - LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); - - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } - - /// Update the locking node - if (!my_clock.is_stale()) - { - zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); - if (my_clock.clean_state_version) - zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); - else - zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); - } - else - { - LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); - /// clean state is stale - return false; - } - - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); - if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) - zookeeper->set(current_shards_path, host_id); - } - - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); - return true; - } - - - static constexpr UInt64 max_table_tries = 1000; - static constexpr UInt64 max_shard_partition_tries = 600; - - bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) - { - /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint - bool previous_shard_is_instantly_finished = false; - - /// Process each partition that is present in cluster - for (const String & partition_name : task_table.ordered_partition_names) - { - if (!task_table.cluster_partitions.count(partition_name)) - throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); - - ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; - - Stopwatch watch; - /// We will check all the shards of the table and check if they contain current partition. - TasksShard expected_shards; - UInt64 num_failed_shards = 0; - - ++cluster_partition.total_tries; - - LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); - - /// Process each source shard having current partition and copy current partition - /// NOTE: shards are sorted by "distance" to current host - bool has_shard_to_process = false; - for (const TaskShardPtr & shard : task_table.all_shards) - { - /// Does shard have a node with current partition? - if (shard->partition_tasks.count(partition_name) == 0) - { - /// If not, did we check existence of that partition previously? - if (shard->checked_partitions.count(partition_name) == 0) - { - auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; - bool has_partition = retry(check_shard_has_partition); - - shard->checked_partitions.emplace(partition_name); - - if (has_partition) - { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name)); - LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); - } - else - { - LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); - continue; - } - } - else - { - /// We have already checked that partition, but did not discover it - previous_shard_is_instantly_finished = true; - continue; - } - } - - auto it_shard_partition = shard->partition_tasks.find(partition_name); - /// Previously when we discovered that shard does not contain current partition, we skipped it. - /// At this moment partition have to be present. - if (it_shard_partition == shard->partition_tasks.end()) - throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); - auto & partition = it_shard_partition->second; - - expected_shards.emplace_back(shard); - - /// Do not sleep if there is a sequence of already processed shards to increase startup - bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; - bool was_error = false; - has_shard_to_process = true; - for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) - { - task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); - - /// Exit if success - if (task_status == PartitionTaskStatus::Finished) - break; - - was_error = true; - - /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) - break; - - /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); - } - - if (task_status == PartitionTaskStatus::Error) - ++num_failed_shards; - - previous_shard_is_instantly_finished = !was_error; - } - - cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); - - /// Check that whole cluster partition is done - /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; - try - { - partition_is_done = - !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); - } - catch (...) - { - tryLogCurrentException(log); - partition_is_done = false; - } - - if (partition_is_done) - { - task_table.finished_cluster_partitions.emplace(partition_name); - - task_table.bytes_copied += cluster_partition.bytes_copied; - task_table.rows_copied += cluster_partition.rows_copied; - double elapsed = cluster_partition.elapsed_time_seconds; - - LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name - << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" - << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" - << " and " << cluster_partition.blocks_copied << " source blocks are copied"); - - if (cluster_partition.rows_copied) - { - LOG_INFO(log, "Average partition speed: " - << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); - } - - if (task_table.rows_copied) - { - LOG_INFO(log, "Average table " << task_table.table_id << " speed: " - << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); - } - } - } - - UInt64 required_partitions = task_table.cluster_partitions.size(); - UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); - bool table_is_done = finished_partitions >= required_partitions; - - if (!table_is_done) - { - LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." - << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); - } - - return table_is_done; - } - - - /// Execution status of a task - enum class PartitionTaskStatus - { - Active, - Finished, - Error, - }; - - /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - PartitionTaskStatus res; - - try - { - res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; - } - - /// At the end of each task check if the config is updated - try - { - updateConfigIfNeeded(); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while updating the config"); - } - - return res; - } - - PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - TaskShard & task_shard = task_partition.task_shard; - TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - const size_t number_of_splits = task_table.number_of_splits; - UNUSED(number_of_splits); - - /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard); - - auto zookeeper = context.getZooKeeper(); - - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - const String current_task_is_active_path = task_partition.getActiveWorkerPath(); - const String current_task_status_path = task_partition.getShardStatusPath(); - - /// Auxiliary functions: - - /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) - { - if (clock.is_stale()) - LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); - else if (!clock.is_clean()) - LOG_DEBUG(log, "Thank you, Captain Obvious"); - else if (clock.discovery_version) - { - LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); - } - else - { - LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); - } - }; - - /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false, size_t current_piece_number = 0) - { - String query; - query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); - /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) - query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - if (enable_splitting) - query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - if (!limit.empty()) - query += " LIMIT " + limit; - - ParserQuery p_query(query.data() + query.size()); - return parseQuery(p_query, query, 0); - }; - - /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); - - LOG_DEBUG(log, "Processing " << current_task_status_path); - - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - LogicalClock task_start_clock; - { - Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - } - - /// Do not start if partition is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_status_path); - } - else - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); - - try - { - tryDropPartition(task_partition, zookeeper, clean_state_clock); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred when clean partition"); - } - - return PartitionTaskStatus::Error; - } - - /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_is_active_path); - zkutil::EphemeralNodeHolderPtr partition_task_node_holder; - try - { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); - return PartitionTaskStatus::Active; - } - - throw; - } - - /// Exit if task has been already processed; - /// create blocking node to signal cleaning up if it is abandoned - { - String status_data; - if (zookeeper->tryGet(current_task_status_path, status_data)) - { - TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); - if (status.state == TaskState::Finished) - { - LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; - } - - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - - /// Check that destination partition is empty if we are first worker - /// NOTE: this check is incorrect if pull and push tables have different partition key! - String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") - { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", - *zookeeper, host_id); - // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); - UInt64 count; - { - Context local_context = context; - // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.getSettingsRef() = task_cluster->settings_pull; - local_context.getSettingsRef().skip_unavailable_shards = true; - - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); - count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; - } - - if (count != 0) - { - Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); - - /// NOTE: partition is still fresh if dirt discovery happens before cleaning - if (stat_shards.numChildren == 0) - { - LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); - } - /// At this point, we need to sync that the destination table is clean - /// before any actual work - - /// Try start processing, create node about it - { - String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); - } - - /// Try create table (if not exists) on each shard - { - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - task_table.table_push, task_table.engine_push_ast); - create_query_push_ast->as().if_not_exists = true; - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); - } - - /// Do the copying - { - bool inject_fault = false; - if (copy_fault_probability > 0) - { - double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); - inject_fault = value < copy_fault_probability; - } - - // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); - - LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() - << " : " << queryToString(query_select_ast)); - - ASTPtr query_insert_ast; - { - String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; - - ParserQuery p_query(query.data() + query.size()); - query_insert_ast = parseQuery(p_query, query, 0); - - LOG_DEBUG(log, "Executing INSERT query: " << query); - } - - try - { - /// Custom INSERT SELECT implementation - Context context_select = context; - context_select.getSettingsRef() = task_cluster->settings_pull; - - Context context_insert = context; - context_insert.getSettingsRef() = task_cluster->settings_push; - - BlockInputStreamPtr input; - BlockOutputStreamPtr output; - { - BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - - input = io_select.in; - output = io_insert.out; - } - - /// Fail-fast optimization to abort copying when the current clean state expires - std::future future_is_dirty_checker; - - Stopwatch watch(CLOCK_MONOTONIC_COARSE); - constexpr UInt64 check_period_milliseconds = 500; - - /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data - auto cancel_check = [&] () - { - if (zookeeper->expired()) - throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - - if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); - - /// check_period_milliseconds should less than average insert time of single block - /// Otherwise, the insertion will slow a little bit - if (watch.elapsedMilliseconds() >= check_period_milliseconds) - { - Coordination::ExistsResponse status = future_is_dirty_checker.get(); - - if (status.error != Coordination::ZNONODE) - { - LogicalClock dirt_discovery_epoch (status.stat.mzxid); - if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) - return false; - throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - } - } - - return false; - }; - - /// Update statistics - /// It is quite rough: bytes_copied don't take into account DROP PARTITION. - auto update_stats = [&cluster_partition] (const Block & block) - { - cluster_partition.bytes_copied += block.bytes(); - cluster_partition.rows_copied += block.rows(); - cluster_partition.blocks_copied += 1; - }; - - /// Main work is here - copyData(*input, *output, cancel_check, update_stats); - - // Just in case - if (future_is_dirty_checker.valid()) - future_is_dirty_checker.get(); - - if (inject_fault) - throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); - return PartitionTaskStatus::Error; - } - } - - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) - { - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->set(current_task_status_path, state_finished, 0); - } - - LOG_INFO(log, "Partition " << task_partition.name << " copied"); - return PartitionTaskStatus::Finished; - } - - void dropAndCreateLocalTable(const ASTPtr & create_ast) - { - const auto & create = create_ast->as(); - dropLocalTableIfExists({create.database, create.table}); - - InterpreterCreateQuery interpreter(create_ast, context); - interpreter.execute(); - } - - void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const - { - auto drop_ast = std::make_shared(); - drop_ast->if_exists = true; - drop_ast->database = table_name.first; - drop_ast->table = table_name.second; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - - String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) - { - String query = "SHOW CREATE TABLE " + getQuotedTable(table); - Block block = getBlockWithAllStreamData(std::make_shared( - connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); - - return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); - } - - ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - /// Fetch and parse (possibly) new definition - auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); - String create_query_pull_str = getRemoteCreateTable( - task_shard.task_table.table_pull, - *connection_entry, - &task_cluster->settings_pull); - - ParserCreateQuery parser_create_query; - return parseQuery(parser_create_query, create_query_pull_str, 0); - } - - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) - { - TaskTable & task_table = task_shard.task_table; - - /// We need to update table definitions for each part, it could be changed after ALTER - task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); - - /// Create local Distributed tables: - /// a table fetching data from current shard and a table inserting data to the whole destination cluster - String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; - String split_shard_prefix = ".split."; - task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); - - /// Create special cluster with single shard - String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; - ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); - context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); - - auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_split_ast = task_table.engine_split_ast; - - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); - - dropAndCreateLocalTable(create_table_pull_ast); - - if (create_split) - dropAndCreateLocalTable(create_table_split_ast); - } - - - std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" - << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; - query = wb.str(); - } - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); - - std::set res; - if (block) - { - ColumnWithTypeAndName & column = block.getByPosition(0); - task_shard.partition_key_column = column; - - for (size_t i = 0; i < column.column->size(); ++i) - { - WriteBufferFromOwnString wb; - column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); - res.emplace(wb.str()); - } - } - - LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); - - return res; - } - - bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; - } - - /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster - * Returns number of shards for which at least one replica executed query successfully - */ - UInt64 executeQueryOnCluster( - const ClusterPtr & cluster, - const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, - PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const - { - auto num_shards = cluster->getShardsInfo().size(); - std::vector per_shard_num_successful_replicas(num_shards, 0); - - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, 0); - } - else - query_ast = query_ast_; - - - /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) - { - const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); - UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); - num_successful_executions = 0; - - auto increment_and_check_exit = [&] () -> bool - { - ++num_successful_executions; - return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; - }; - - UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - UInt64 num_local_replicas = shard.getLocalNodeCount(); - UInt64 num_remote_replicas = num_replicas - num_local_replicas; - - /// In that case we don't have local replicas, but do it just in case - for (UInt64 i = 0; i < num_local_replicas; ++i) - { - auto interpreter = InterpreterFactory::get(query_ast, context); - interpreter->execute(); - - if (increment_and_check_exit()) - return; - } - - /// Will try to make as many as possible queries - if (shard.hasRemoteConnections()) - { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); - - for (auto & connection : connections) - { - if (connection.isNull()) - continue; - - try - { - /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; - NullBlockOutputStream output{Block{}}; - copyData(stream, output); - - if (increment_and_check_exit()) - return; - } - catch (const Exception &) - { - LOG_INFO(log, getCurrentExceptionMessage(false, true)); - } - } - } - }; - - { - ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); - - for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); - - thread_pool.wait(); - } - - UInt64 successful_shards = 0; - for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); - - return successful_shards; - } - -private: - String task_zookeeper_path; - String task_description_path; - String host_id; - String working_database_name; - - /// Auto update config stuff - UInt64 task_descprtion_current_version = 1; - std::atomic task_descprtion_version{1}; - Coordination::WatchCallback task_description_watch_callback; - /// ZooKeeper session used to set the callback - zkutil::ZooKeeperPtr task_description_watch_zookeeper; - - ConfigurationPtr task_cluster_initial_config; - ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_description_current_stat{}; - - std::unique_ptr task_cluster; - - bool is_safe_mode = false; - double copy_fault_probability = 0.0; - - Context & context; - Poco::Logger * log; - - std::chrono::milliseconds default_sleep_time{1000}; -}; - - -/// ClusterCopierApp - - -void ClusterCopierApp::initialize(Poco::Util::Application & self) -{ - is_help = config().has("help"); - if (is_help) - return; - - config_xml_path = config().getString("config-file"); - task_path = config().getString("task-path"); - log_level = config().getString("log-level", "trace"); - is_safe_mode = config().has("safe-mode"); - if (config().has("copy-fault-probability")) - copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); - base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); - // process_id is '#_' - time_t timestamp = Poco::Timestamp().epochTime(); - auto curr_pid = Poco::Process::id(); - - process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); - host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); - Poco::File(process_path).createDirectories(); - - /// Override variables for BaseDaemon - if (config().has("log-level")) - config().setString("logger.level", config().getString("log-level")); - - if (config().has("base-dir") || !config().has("logger.log")) - config().setString("logger.log", process_path + "/log.log"); - - if (config().has("base-dir") || !config().has("logger.errorlog")) - config().setString("logger.errorlog", process_path + "/log.err.log"); - - Base::initialize(self); -} - - -void ClusterCopierApp::handleHelp(const std::string &, const std::string &) -{ - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("Copies tables from one cluster to another"); - helpFormatter.setUsage("--config-file --task-path "); - helpFormatter.format(std::cerr); - - stopOptionsProcessing(); -} - - -void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) -{ - Base::defineOptions(options); - - options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper") - .argument("task-path").binding("task-path")); - options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path") - .argument("task-file").binding("task-file")); - options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists") - .argument("task-upload-force").binding("task-upload-force")); - options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors") - .binding("safe-mode")); - options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") - .argument("copy-fault-probability").binding("copy-fault-probability")); - options.addOption(Poco::Util::Option("log-level", "", "sets log level") - .argument("log-level").binding("log-level")); - options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") - .argument("base-dir").binding("base-dir")); - - using Me = std::decay_t; - options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") - .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); -} - - -void ClusterCopierApp::mainImpl() -{ - StatusFile status_file(process_path + "/status"); - ThreadStatus thread_status; - - auto log = &logger(); - LOG_INFO(log, "Starting clickhouse-copier (" - << "id " << process_id << ", " - << "host_id " << host_id << ", " - << "path " << process_path << ", " - << "revision " << ClickHouseRevision::get() << ")"); - - auto context = std::make_unique(Context::createGlobal()); - context->makeGlobalContext(); - SCOPE_EXIT(context->shutdown()); - - context->setConfig(loaded_config.configuration); - context->setApplicationType(Context::ApplicationType::LOCAL); - context->setPath(process_path); - - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - registerStorages(); - registerDictionaries(); - registerDisks(); - - static const std::string default_database = "_local"; - context->addDatabase(default_database, std::make_shared(default_database)); - context->setCurrentDatabase(default_database); - - /// Initialize query scope just in case. - CurrentThread::QueryScope query_scope(*context); - - auto copier = std::make_unique(task_path, host_id, default_database, *context); - copier->setSafeMode(is_safe_mode); - copier->setCopyFaultProbability(copy_fault_probability); - - auto task_file = config().getString("task-file", ""); - if (!task_file.empty()) - copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); - - copier->init(); - copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); - - /// Reset ZooKeeper before removing ClusterCopier. - /// Otherwise zookeeper watch can call callback which use already removed ClusterCopier object. - context->resetZooKeeper(); -} - - -int ClusterCopierApp::main(const std::vector &) -{ - if (is_help) - return 0; - - try - { - mainImpl(); - } - catch (...) - { - tryLogCurrentException(&Poco::Logger::root(), __PRETTY_FUNCTION__); - auto code = getCurrentExceptionCode(); - - return (code) ? code : -1; - } - - return 0; -} - - -} - -#pragma GCC diagnostic ignored "-Wunused-function" -#pragma GCC diagnostic ignored "-Wmissing-declarations" - -int mainEntryClickHouseClusterCopier(int argc, char ** argv) -{ - try - { - DB::ClusterCopierApp app; - return app.run(argc, argv); - } - catch (...) - { - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - auto code = DB::getCurrentExceptionCode(); - - return (code) ? code : -1; - } -} diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index fe228fd6194..f4f37b97c26 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,87 +1,1525 @@ #pragma once -#include -#include -/* clickhouse cluster copier util - * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. - * - * See overview in the docs: docs/en/utils/clickhouse-copier.md - * - * Implementation details: - * - * cluster-copier workers pull each partition of each shard of the source cluster and push it to the destination cluster through - * Distributed table (to preform data resharding). So, worker job is a partition of a source shard. - * A job has three states: Active, Finished and Abandoned. Abandoned means that worker died and did not finish the job. - * - * If an error occurred during the copying (a worker failed or a worker did not finish the INSERT), then the whole partition (on - * all destination servers) should be dropped and refilled. So, copying entity is a partition of all destination shards. - * If a failure is detected a special /is_dirty node is created in ZooKeeper signalling that other workers copying the same partition - * should stop, after a refilling procedure should start. - * - * ZooKeeper task node has the following structure: - * /task/path_root - path passed in --task-path parameter - * /description - contains user-defined XML config of the task - * /task_active_workers - contains ephemeral nodes of all currently active workers, used to implement max_workers limitation - * /server_fqdn#PID_timestamp - cluster-copier worker ID - * ... - * /tables - directory with table tasks - * /cluster.db.table1 - directory of table_hits task - * /partition1 - directory for partition1 - * /shards - directory for source cluster shards - * /1 - worker job for the first shard of partition1 of table test.hits - * Contains info about current status (Active or Finished) and worker ID. - * /2 - * ... - * /partition_active_workers - * /1 - for each job in /shards a corresponding ephemeral node created in /partition_active_workers - * It is used to detect Abandoned jobs (if there is Active node in /shards and there is no node in - * /partition_active_workers). - * Also, it is used to track active workers in the partition (when we need to refill the partition we do - * not DROP PARTITION while there are active workers) - * /2 - * ... - * /is_dirty - the node is set if some worker detected that an error occurred (the INSERT is failed or an Abandoned node is - * detected). If the node appeared workers in this partition should stop and start cleaning and refilling - * partition procedure. - * During this procedure a single 'cleaner' worker is selected. The worker waits for stopping all partition - * workers, removes /shards node, executes DROP PARTITION on each destination node and removes /is_dirty node. - * /cleaner- An ephemeral node used to select 'cleaner' worker. Contains ID of the worker. - * /cluster.db.table2 - * ... - */ +#include "Internals.h" +#include "ZookeeperStaff.h" namespace DB { -class ClusterCopierApp : public BaseDaemon +using ConfigurationPtr = Poco::AutoPtr; + +static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) +{ + std::stringstream ss(xml_data); + Poco::XML::InputSource input_source{ss}; + return {new Poco::Util::XMLConfiguration{&input_source}}; +} + +class ClusterCopier { public: - void initialize(Poco::Util::Application & self) override; + ClusterCopier(String task_path_, + String host_id_, + String proxy_database_name_, + Context & context_) + : + task_zookeeper_path(std::move(task_path_)), + host_id(std::move(host_id_)), + working_database_name(std::move(proxy_database_name_)), + context(context_), + log(&Poco::Logger::get("ClusterCopier")) + { + } - void handleHelp(const std::string &, const std::string &); + void init() + { + auto zookeeper = context.getZooKeeper(); - void defineOptions(Poco::Util::OptionSet & options) override; + task_description_watch_callback = [this] (const Coordination::WatchResponse & response) + { + if (response.error != Coordination::ZOK) + return; + UInt64 version = ++task_descprtion_version; + LOG_DEBUG(log, "Task description should be updated, local version " << version); + }; - int main(const std::vector &) override; + task_description_path = task_zookeeper_path + "/description"; + task_cluster = std::make_unique(task_zookeeper_path, working_database_name); + + reloadTaskDescription(); + task_cluster_initial_config = task_cluster_current_config; + + task_cluster->loadTasks(*task_cluster_initial_config); + context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + + /// Set up shards and their priority + task_cluster->random_engine.seed(task_cluster->random_device()); + for (auto & task_table : task_cluster->table_tasks) + { + task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + task_table.cluster_push = context.getCluster(task_table.cluster_push_name); + task_table.initShards(task_cluster->random_engine); + } + + LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); + + /// Do not initialize tables, will make deferred initialization in process() + + zookeeper->createAncestors(getWorkersPathVersion() + "/"); + zookeeper->createAncestors(getWorkersPath() + "/"); + } + + template + decltype(auto) retry(T && func, UInt64 max_tries = 100) + { + std::exception_ptr exception; + + for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) + { + try + { + return func(); + } + catch (...) + { + exception = std::current_exception(); + if (try_number < max_tries) + { + tryLogCurrentException(log, "Will retry"); + std::this_thread::sleep_for(default_sleep_time); + } + } + } + + std::rethrow_exception(exception); + } + + + void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) + { + TaskTable & task_table = task_shard->task_table; + + LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); + + auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; + auto existing_partitions_names = retry(get_partitions, 60); + Strings filtered_partitions_names; + Strings missing_partitions; + + /// Check that user specified correct partition names + auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) + { + MutableColumnPtr column_dummy = type->createColumn(); + ReadBufferFromString rb(partition_text_quoted); + + try + { + type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); + } + catch (Exception & e) + { + throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); + } + }; + + if (task_table.has_enabled_partitions) + { + /// Process partition in order specified by + for (const String & partition_name : task_table.enabled_partitions) + { + /// Check that user specified correct partition names + check_partition_format(task_shard->partition_key_column.type, partition_name); + + auto it = existing_partitions_names.find(partition_name); + + /// Do not process partition if it is not in enabled_partitions list + if (it == existing_partitions_names.end()) + { + missing_partitions.emplace_back(partition_name); + continue; + } + + filtered_partitions_names.emplace_back(*it); + } + + for (const String & partition_name : existing_partitions_names) + { + if (!task_table.enabled_partitions_set.count(partition_name)) + { + LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " + << "enabled_partitions of " << task_table.table_id); + } + } + } + else + { + for (const String & partition_name : existing_partitions_names) + filtered_partitions_names.emplace_back(partition_name); + } + + for (const String & partition_name : filtered_partitions_names) + { + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + task_shard->checked_partitions.emplace(partition_name, true); + } + + if (!missing_partitions.empty()) + { + std::stringstream ss; + for (const String & missing_partition : missing_partitions) + ss << " " << missing_partition; + + LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " + << task_shard->getDescription() << " :" << ss.str()); + } + + LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); + } + + /// Compute set of partitions, assume set of partitions aren't changed during the processing + void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) + { + /// Fetch partitions list from a shard + { + ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + + for (const TaskShardPtr & task_shard : task_table.all_shards) + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + + LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); + thread_pool.wait(); + } + } + + void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) + { + auto local_task_description_path = task_path + "/description"; + + String task_config_str; + { + ReadBufferFromFile in(task_file); + readStringUntilEOF(task_config_str, in); + } + if (task_config_str.empty()) + return; + + auto zookeeper = context.getZooKeeper(); + + zookeeper->createAncestors(local_task_description_path); + auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + if (code && force) + zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + + LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); + } + + void reloadTaskDescription() + { + auto zookeeper = context.getZooKeeper(); + task_description_watch_zookeeper = zookeeper; + + String task_config_str; + Coordination::Stat stat{}; + int code; + + zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); + if (code) + throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); + + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); + auto config = getConfigurationFromXMLString(task_config_str); + + /// Setup settings + task_cluster->reloadSettings(*config); + context.getSettingsRef() = task_cluster->settings_common; + + task_cluster_current_config = config; + task_description_current_stat = stat; + } + + void updateConfigIfNeeded() + { + UInt64 version_to_update = task_descprtion_version; + bool is_outdated_version = task_descprtion_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + + if (!is_outdated_version && !is_expired_session) + return; + + LOG_DEBUG(log, "Updating task description"); + reloadTaskDescription(); + + task_descprtion_current_version = version_to_update; + } + + void process(const ConnectionTimeouts & timeouts) + { + for (TaskTable & task_table : task_cluster->table_tasks) + { + LOG_INFO(log, "Process table task " << task_table.table_id << " with " + << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); + + if (task_table.all_shards.empty()) + continue; + + /// Discover partitions of each shard and total set of partitions + if (!task_table.has_enabled_partitions) + { + /// If there are no specified enabled_partitions, we must discover them manually + discoverTablePartitions(timeouts, task_table); + + /// After partitions of each shard are initialized, initialize cluster partitions + for (const TaskShardPtr & task_shard : task_table.all_shards) + { + for (const auto & partition_elem : task_shard->partition_tasks) + { + const String & partition_name = partition_elem.first; + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + } + } + + for (auto & partition_elem : task_table.cluster_partitions) + { + const String & partition_name = partition_elem.first; + + for (const TaskShardPtr & task_shard : task_table.all_shards) + task_shard->checked_partitions.emplace(partition_name); + + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + else + { + /// If enabled_partitions are specified, assume that each shard has all partitions + /// We will refine partition set of each shard in future + + for (const String & partition_name : task_table.enabled_partitions) + { + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + + task_table.watch.restart(); + + /// Retry table processing + bool table_is_done = false; + for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) + { + if (tryProcessTable(timeouts, task_table)) + { + table_is_done = true; + break; + } + } + + if (!table_is_done) + { + throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", + ErrorCodes::UNFINISHED); + } + } + } + + /// Disables DROP PARTITION commands that used to clear data after errors + void setSafeMode(bool is_safe_mode_ = true) + { + is_safe_mode = is_safe_mode_; + } + + void setCopyFaultProbability(double copy_fault_probability_) + { + copy_fault_probability = copy_fault_probability_; + } + + + protected: + + String getWorkersPath() const + { + return task_cluster->task_zookeeper_path + "/task_active_workers"; + } + + String getWorkersPathVersion() const + { + return getWorkersPath() + "_version"; + } + + String getCurrentWorkerNodePath() const + { + return getWorkersPath() + "/" + host_id; + } + + zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( + const zkutil::ZooKeeperPtr & zookeeper, + const String & description, + bool unprioritized) + { + std::chrono::milliseconds current_sleep_time = default_sleep_time; + static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec + + if (unprioritized) + std::this_thread::sleep_for(current_sleep_time); + + String workers_version_path = getWorkersPathVersion(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); + + UInt64 num_bad_version_errors = 0; + + while (true) + { + updateConfigIfNeeded(); + + Coordination::Stat stat{}; + zookeeper->get(workers_version_path, &stat); + auto version = stat.version; + zookeeper->get(workers_path, &stat); + + if (static_cast(stat.numChildren) >= task_cluster->max_workers) + { + LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" + << ". Postpone processing " << description); + + if (unprioritized) + current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); + + std::this_thread::sleep_for(current_sleep_time); + num_bad_version_errors = 0; + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); + ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + + if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) + return std::make_shared(current_worker_path, *zookeeper, false, false, description); + + if (code == Coordination::ZBADVERSION) + { + ++num_bad_version_errors; + + /// Try to make fast retries + if (num_bad_version_errors > 3) + { + LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); + std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); + std::this_thread::sleep_for(random_sleep_time); + num_bad_version_errors = 0; + } + } + else + throw Coordination::Exception(code); + } + } + } + + /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. + * State of some task could change during the processing. + * We have to ensure that all shards have the finished state and there is no dirty flag. + * Moreover, we have to check status twice and check zxid, because state can change during the checking. + */ + bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + { + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + + auto zookeeper = context.getZooKeeper(); + + Strings status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getPartitionIsDirtyPath(partition_name), + task_table.getPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name + << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + } + + /// Removes MATERIALIZED and ALIAS columns from create table query + static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) + { + const ASTs & column_asts = query_ast->as().columns_list->columns->children; + auto new_columns = std::make_shared(); + + for (const ASTPtr & column_ast : column_asts) + { + const auto & column = column_ast->as(); + + if (!column.default_specifier.empty()) + { + ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); + if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) + continue; + } + + new_columns->children.emplace_back(column_ast->clone()); + } + + ASTPtr new_query_ast = query_ast->clone(); + auto & new_query = new_query_ast->as(); + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, new_columns); + if (auto indices = query_ast->as()->columns_list->indices) + new_columns_list->set(new_columns_list->indices, indices->clone()); + + new_query.replace(new_query.columns_list, new_columns_list); + + return new_query_ast; + } + + /// Replaces ENGINE and table name in a create query + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) + { + const auto & create = create_query_ast->as(); + auto res = std::make_shared(create); + + if (create.storage == nullptr || new_storage_ast == nullptr) + throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); + + res->database = new_table.first; + res->table = new_table.second; + + res->children.clear(); + res->set(res->columns_list, create.columns_list->clone()); + res->set(res->storage, new_storage_ast->clone()); + + return res; + } + + bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) + { + if (is_safe_mode) + throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); + + TaskTable & task_table = task_partition.task_shard.task_table; + + const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + + zkutil::EphemeralNodeHolder::Ptr cleaner_holder; + try + { + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + + throw; + } + + Coordination::Stat stat{}; + if (zookeeper->exists(current_partition_active_workers_dir, &stat)) + { + if (stat.numChildren != 0) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + else + { + zookeeper->remove(current_partition_active_workers_dir); + } + } + + { + zkutil::EphemeralNodeHolder::Ptr active_workers_lock; + try + { + active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); + return false; + } + + throw; + } + + // Lock the dirty flag + zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); + zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + /// Remove all status nodes + { + Strings children; + if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) + for (const auto & child : children) + { + zookeeper->removeRecursive(current_shards_path + "/" + child); + } + } + + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + query += " DROP PARTITION " + task_partition.name + ""; + + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + + ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + /// It is important, DROP PARTITION must be done synchronously + settings_push.replication_alter_partitions_sync = 2; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// Limit number of max executing replicas to 1 + UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + + if (num_shards < cluster_push->getShardCount()) + { + LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); + return false; + } + + /// Update the locking node + if (!my_clock.is_stale()) + { + zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); + if (my_clock.clean_state_version) + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); + else + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); + } + else + { + LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + /// clean state is stale + return false; + } + + LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) + zookeeper->set(current_shards_path, host_id); + } + + LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + return true; + } + + + static constexpr UInt64 max_table_tries = 1000; + static constexpr UInt64 max_shard_partition_tries = 600; + + bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) + { + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + bool previous_shard_is_instantly_finished = false; + + /// Process each partition that is present in cluster + for (const String & partition_name : task_table.ordered_partition_names) + { + if (!task_table.cluster_partitions.count(partition_name)) + throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); + + ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; + + Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. + TasksShard expected_shards; + UInt64 num_failed_shards = 0; + + ++cluster_partition.total_tries; + + LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); + + /// Process each source shard having current partition and copy current partition + /// NOTE: shards are sorted by "distance" to current host + bool has_shard_to_process = false; + for (const TaskShardPtr & shard : task_table.all_shards) + { + /// Does shard have a node with current partition? + if (shard->partition_tasks.count(partition_name) == 0) + { + /// If not, did we check existence of that partition previously? + if (shard->checked_partitions.count(partition_name) == 0) + { + auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; + bool has_partition = retry(check_shard_has_partition); + + shard->checked_partitions.emplace(partition_name); + + if (has_partition) + { + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); + /// To save references in the future. + auto shard_partition_it = shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + ///FIXME: Remove 10 + for (int piece_number = 0; piece_number < 10; ++piece_number) + { + auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + else + { + LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); + continue; + } + } + else + { + /// We have already checked that partition, but did not discover it + previous_shard_is_instantly_finished = true; + continue; + } + } + + auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. + if (it_shard_partition == shard->partition_tasks.end()) + throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); + auto & partition = it_shard_partition->second; + + expected_shards.emplace_back(shard); + + /// Do not sleep if there is a sequence of already processed shards to increase startup + bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; + PartitionTaskStatus task_status = PartitionTaskStatus::Error; + bool was_error = false; + has_shard_to_process = true; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); + + /// Exit if success + if (task_status == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (task_status == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + if (task_status == PartitionTaskStatus::Error) + ++num_failed_shards; + + previous_shard_is_instantly_finished = !was_error; + } + + cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); + + /// Check that whole cluster partition is done + /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done + bool partition_is_done = num_failed_shards == 0; + try + { + partition_is_done = + !has_shard_to_process + || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + } + catch (...) + { + tryLogCurrentException(log); + partition_is_done = false; + } + + if (partition_is_done) + { + task_table.finished_cluster_partitions.emplace(partition_name); + + task_table.bytes_copied += cluster_partition.bytes_copied; + task_table.rows_copied += cluster_partition.rows_copied; + double elapsed = cluster_partition.elapsed_time_seconds; + + LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name + << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" + << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" + << " and " << cluster_partition.blocks_copied << " source blocks are copied"); + + if (cluster_partition.rows_copied) + { + LOG_INFO(log, "Average partition speed: " + << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); + } + + if (task_table.rows_copied) + { + LOG_INFO(log, "Average table " << task_table.table_id << " speed: " + << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); + } + } + } + + UInt64 required_partitions = task_table.cluster_partitions.size(); + UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); + bool table_is_done = finished_partitions >= required_partitions; + + if (!table_is_done) + { + LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." + << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); + } + + return table_is_done; + } + + + /// Execution status of a task + enum class PartitionTaskStatus + { + Active, + Finished, + Error, + }; + + /// Job for copying partition from particular shard. + PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + { + PartitionTaskStatus res; + + try + { + res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); + res = PartitionTaskStatus::Error; + } + + /// At the end of each task check if the config is updated + try + { + updateConfigIfNeeded(); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while updating the config"); + } + + return res; + } + + PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + { + TaskShard & task_shard = task_partition.task_shard; + TaskTable & task_table = task_shard.task_table; + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + const size_t number_of_splits = task_table.number_of_splits; + UNUSED(number_of_splits); + + /// We need to update table definitions for each partition, it could be changed after ALTER + createShardInternalTables(timeouts, task_shard); + + auto zookeeper = context.getZooKeeper(); + + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String current_task_is_active_path = task_partition.getActiveWorkerPath(); + const String current_task_status_path = task_partition.getShardStatusPath(); + + /// Auxiliary functions: + + /// Creates is_dirty node to initialize DROP PARTITION + auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + { + if (clock.is_stale()) + LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + else if (!clock.is_clean()) + LOG_DEBUG(log, "Thank you, Captain Obvious"); + else if (clock.discovery_version) + { + LOG_DEBUG(log, "Updating clean state clock"); + zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); + } + else + { + LOG_DEBUG(log, "Creating clean state clock"); + zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + } + }; + + /// Returns SELECT query filtering current partition and applying user filter + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false, size_t current_piece_number = 0) + { + String query; + query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); + /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) + query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) + query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + if (!limit.empty()) + query += " LIMIT " + limit; + + ParserQuery p_query(query.data() + query.size()); + return parseQuery(p_query, query, 0); + }; + + /// Load balancing + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); + + LOG_DEBUG(log, "Processing " << current_task_status_path); + + CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + LogicalClock task_start_clock; + { + Coordination::Stat stat{}; + if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + } + + /// Do not start if partition is dirty, try to clean it + if (clean_state_clock.is_clean() + && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + zookeeper->createAncestors(current_task_status_path); + } + else + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + + try + { + tryDropPartition(task_partition, zookeeper, clean_state_clock); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred when clean partition"); + } + + return PartitionTaskStatus::Error; + } + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_task_is_active_path); + zkutil::EphemeralNodeHolderPtr partition_task_node_holder; + try + { + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); + return PartitionTaskStatus::Active; + } + + throw; + } + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_task_status_path, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); + return PartitionTaskStatus::Finished; + } + + // Task is abandoned, initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + + /// Check that destination partition is empty if we are first worker + /// NOTE: this check is incorrect if pull and push tables have different partition key! + String clean_start_status; + if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + { + zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); + // Maybe we are the first worker + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + UInt64 count; + { + Context local_context = context; + // Use pull (i.e. readonly) settings, but fetch data from destination servers + local_context.getSettingsRef() = task_cluster->settings_pull; + local_context.getSettingsRef().skip_unavailable_shards = true; + + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; + } + + if (count != 0) + { + Coordination::Stat stat_shards{}; + zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); + + /// NOTE: partition is still fresh if dirt discovery happens before cleaning + if (stat_shards.numChildren == 0) + { + LOG_WARNING(log, "There are no workers for partition " << task_partition.name + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + } + /// At this point, we need to sync that the destination table is clean + /// before any actual work + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); + } + + /// Try create table (if not exists) on each shard + { + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Do the copying + { + bool inject_fault = false; + if (copy_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < copy_fault_probability; + } + + // Select all fields + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + + LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() + << " : " << queryToString(query_select_ast)); + + ASTPtr query_insert_ast; + { + String query; + query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + + ParserQuery p_query(query.data() + query.size()); + query_insert_ast = parseQuery(p_query, query, 0); + + LOG_DEBUG(log, "Executing INSERT query: " << query); + } + + try + { + /// Custom INSERT SELECT implementation + Context context_select = context; + context_select.getSettingsRef() = task_cluster->settings_pull; + + Context context_insert = context; + context_insert.getSettingsRef() = task_cluster->settings_push; + + BlockInputStreamPtr input; + BlockOutputStreamPtr output; + { + BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); + BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); + + input = io_select.in; + output = io_insert.out; + } + + /// Fail-fast optimization to abort copying when the current clean state expires + std::future future_is_dirty_checker; + + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + constexpr UInt64 check_period_milliseconds = 500; + + /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data + auto cancel_check = [&] () + { + if (zookeeper->expired()) + throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + + if (!future_is_dirty_checker.valid()) + future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); + + /// check_period_milliseconds should less than average insert time of single block + /// Otherwise, the insertion will slow a little bit + if (watch.elapsedMilliseconds() >= check_period_milliseconds) + { + Coordination::ExistsResponse status = future_is_dirty_checker.get(); + + if (status.error != Coordination::ZNONODE) + { + LogicalClock dirt_discovery_epoch (status.stat.mzxid); + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) + return false; + throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + } + } + + return false; + }; + + /// Update statistics + /// It is quite rough: bytes_copied don't take into account DROP PARTITION. + auto update_stats = [&cluster_partition] (const Block & block) + { + cluster_partition.bytes_copied += block.bytes(); + cluster_partition.rows_copied += block.rows(); + cluster_partition.blocks_copied += 1; + }; + + /// Main work is here + copyData(*input, *output, cancel_check, update_stats); + + // Just in case + if (future_is_dirty_checker.valid()) + future_is_dirty_checker.get(); + + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + return PartitionTaskStatus::Error; + } + } + + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->set(current_task_status_path, state_finished, 0); + } + + LOG_INFO(log, "Partition " << task_partition.name << " copied"); + return PartitionTaskStatus::Finished; + } + + void dropAndCreateLocalTable(const ASTPtr & create_ast) + { + const auto & create = create_ast->as(); + dropLocalTableIfExists({create.database, create.table}); + + InterpreterCreateQuery interpreter(create_ast, context); + interpreter.execute(); + } + + void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const + { + auto drop_ast = std::make_shared(); + drop_ast->if_exists = true; + drop_ast->database = table_name.first; + drop_ast->table = table_name.second; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); + } + + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) + { + String query = "SHOW CREATE TABLE " + getQuotedTable(table); + Block block = getBlockWithAllStreamData(std::make_shared( + connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); + + return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); + } + + ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) + { + /// Fetch and parse (possibly) new definition + auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); + String create_query_pull_str = getRemoteCreateTable( + task_shard.task_table.table_pull, + *connection_entry, + &task_cluster->settings_pull); + + ParserCreateQuery parser_create_query; + return parseQuery(parser_create_query, create_query_pull_str, 0); + } + + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) + { + TaskTable & task_table = task_shard.task_table; + + /// We need to update table definitions for each part, it could be changed after ALTER + task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + + /// Create local Distributed tables: + /// a table fetching data from current shard and a table inserting data to the whole destination cluster + String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; + String split_shard_prefix = ".split."; + task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + + /// Create special cluster with single shard + String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; + ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); + context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); + + auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); + const auto & storage_split_ast = task_table.engine_split_ast; + + auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); + auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); + + dropAndCreateLocalTable(create_table_pull_ast); + + if (create_split) + dropAndCreateLocalTable(create_table_split_ast); + } + + + std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + String query; + { + WriteBufferFromOwnString wb; + wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" + << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; + query = wb.str(); + } + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + + std::set res; + if (block) + { + ColumnWithTypeAndName & column = block.getByPosition(0); + task_shard.partition_key_column = column; + + for (size_t i = 0; i < column.column->size(); ++i) + { + WriteBufferFromOwnString wb; + column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); + res.emplace(wb.str()); + } + } + + LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); + + return res; + } + + bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + } + + /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); + /// Just copypaste the function above + bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + query += " AND (cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + } + + /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster + * Returns number of shards for which at least one replica executed query successfully + */ + UInt64 executeQueryOnCluster( + const ClusterPtr & cluster, + const String & query, + const ASTPtr & query_ast_ = nullptr, + const Settings * settings = nullptr, + PoolMode pool_mode = PoolMode::GET_ALL, + UInt64 max_successful_executions_per_shard = 0) const + { + auto num_shards = cluster->getShardsInfo().size(); + std::vector per_shard_num_successful_replicas(num_shards, 0); + + ASTPtr query_ast; + if (query_ast_ == nullptr) + { + ParserQuery p_query(query.data() + query.size()); + query_ast = parseQuery(p_query, query, 0); + } + else + query_ast = query_ast_; + + + /// We need to execute query on one replica at least + auto do_for_shard = [&] (UInt64 shard_index) + { + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); + UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); + num_successful_executions = 0; + + auto increment_and_check_exit = [&] () -> bool + { + ++num_successful_executions; + return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; + }; + + UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + UInt64 num_local_replicas = shard.getLocalNodeCount(); + UInt64 num_remote_replicas = num_replicas - num_local_replicas; + + /// In that case we don't have local replicas, but do it just in case + for (UInt64 i = 0; i < num_local_replicas; ++i) + { + auto interpreter = InterpreterFactory::get(query_ast, context); + interpreter->execute(); + + if (increment_and_check_exit()) + return; + } + + /// Will try to make as many as possible queries + if (shard.hasRemoteConnections()) + { + Settings current_settings = settings ? *settings : task_cluster->settings_common; + current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + + for (auto & connection : connections) + { + if (connection.isNull()) + continue; + + try + { + /// CREATE TABLE and DROP PARTITION queries return empty block + RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + NullBlockOutputStream output{Block{}}; + copyData(stream, output); + + if (increment_and_check_exit()) + return; + } + catch (const Exception &) + { + LOG_INFO(log, getCurrentExceptionMessage(false, true)); + } + } + } + }; + + { + ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); + + for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) + thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + + thread_pool.wait(); + } + + UInt64 successful_shards = 0; + for (UInt64 num_replicas : per_shard_num_successful_replicas) + successful_shards += (num_replicas > 0); + + return successful_shards; + } private: + String task_zookeeper_path; + String task_description_path; + String host_id; + String working_database_name; - using Base = BaseDaemon; + /// Auto update config stuff + UInt64 task_descprtion_current_version = 1; + std::atomic task_descprtion_version{1}; + Coordination::WatchCallback task_description_watch_callback; + /// ZooKeeper session used to set the callback + zkutil::ZooKeeperPtr task_description_watch_zookeeper; - void mainImpl(); + ConfigurationPtr task_cluster_initial_config; + ConfigurationPtr task_cluster_current_config; + Coordination::Stat task_description_current_stat{}; + + std::unique_ptr task_cluster; - std::string config_xml_path; - std::string task_path; - std::string log_level = "trace"; bool is_safe_mode = false; - double copy_fault_probability = 0; - bool is_help = false; + double copy_fault_probability = 0.0; - std::string base_dir; - std::string process_path; - std::string process_id; - std::string host_id; + Context & context; + Poco::Logger * log; + + std::chrono::milliseconds default_sleep_time{1000}; }; - } diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp new file mode 100644 index 00000000000..37b02296086 --- /dev/null +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -0,0 +1,173 @@ +#include "ClusterCopierApp.h" +#include "ClusterCopier.h" + +namespace DB +{ + +/// ClusterCopierApp + +void ClusterCopierApp::initialize(Poco::Util::Application & self) +{ + is_help = config().has("help"); + if (is_help) + return; + + config_xml_path = config().getString("config-file"); + task_path = config().getString("task-path"); + log_level = config().getString("log-level", "trace"); + is_safe_mode = config().has("safe-mode"); + if (config().has("copy-fault-probability")) + copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); + base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); + // process_id is '#_' + time_t timestamp = Poco::Timestamp().epochTime(); + auto curr_pid = Poco::Process::id(); + + process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); + host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; + process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); + Poco::File(process_path).createDirectories(); + + /// Override variables for BaseDaemon + if (config().has("log-level")) + config().setString("logger.level", config().getString("log-level")); + + if (config().has("base-dir") || !config().has("logger.log")) + config().setString("logger.log", process_path + "/log.log"); + + if (config().has("base-dir") || !config().has("logger.errorlog")) + config().setString("logger.errorlog", process_path + "/log.err.log"); + + Base::initialize(self); +} + + +void ClusterCopierApp::handleHelp(const std::string &, const std::string &) +{ + Poco::Util::HelpFormatter helpFormatter(options()); + helpFormatter.setCommand(commandName()); + helpFormatter.setHeader("Copies tables from one cluster to another"); + helpFormatter.setUsage("--config-file --task-path "); + helpFormatter.format(std::cerr); + + stopOptionsProcessing(); +} + + +void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) +{ + Base::defineOptions(options); + + options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper") + .argument("task-path").binding("task-path")); + options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path") + .argument("task-file").binding("task-file")); + options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists") + .argument("task-upload-force").binding("task-upload-force")); + options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors") + .binding("safe-mode")); + options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") + .argument("copy-fault-probability").binding("copy-fault-probability")); + options.addOption(Poco::Util::Option("log-level", "", "sets log level") + .argument("log-level").binding("log-level")); + options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") + .argument("base-dir").binding("base-dir")); + + using Me = std::decay_t; + options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") + .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); +} + + +void ClusterCopierApp::mainImpl() +{ + StatusFile status_file(process_path + "/status"); + ThreadStatus thread_status; + + auto log = &logger(); + LOG_INFO(log, "Starting clickhouse-copier (" + << "id " << process_id << ", " + << "host_id " << host_id << ", " + << "path " << process_path << ", " + << "revision " << ClickHouseRevision::get() << ")"); + + auto context = std::make_unique(Context::createGlobal()); + context->makeGlobalContext(); + SCOPE_EXIT(context->shutdown()); + + context->setConfig(loaded_config.configuration); + context->setApplicationType(Context::ApplicationType::LOCAL); + context->setPath(process_path); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + registerDictionaries(); + registerDisks(); + + static const std::string default_database = "_local"; + context->addDatabase(default_database, std::make_shared(default_database)); + context->setCurrentDatabase(default_database); + + /// Initialize query scope just in case. + CurrentThread::QueryScope query_scope(*context); + + auto copier = std::make_unique(task_path, host_id, default_database, *context); + copier->setSafeMode(is_safe_mode); + copier->setCopyFaultProbability(copy_fault_probability); + + auto task_file = config().getString("task-file", ""); + if (!task_file.empty()) + copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); + + copier->init(); + copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); + + /// Reset ZooKeeper before removing ClusterCopier. + /// Otherwise zookeeper watch can call callback which use already removed ClusterCopier object. + context->resetZooKeeper(); +} + + +int ClusterCopierApp::main(const std::vector &) +{ + if (is_help) + return 0; + + try + { + mainImpl(); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::root(), __PRETTY_FUNCTION__); + auto code = getCurrentExceptionCode(); + + return (code) ? code : -1; + } + + return 0; +} + + +} + +#pragma GCC diagnostic ignored "-Wunused-function" +#pragma GCC diagnostic ignored "-Wmissing-declarations" + +int mainEntryClickHouseClusterCopier(int argc, char ** argv) +{ + try + { + DB::ClusterCopierApp app; + return app.run(argc, argv); + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; + auto code = DB::getCurrentExceptionCode(); + + return (code) ? code : -1; + } +} diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h new file mode 100644 index 00000000000..fe228fd6194 --- /dev/null +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -0,0 +1,87 @@ +#pragma once +#include +#include + +/* clickhouse cluster copier util + * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. + * + * See overview in the docs: docs/en/utils/clickhouse-copier.md + * + * Implementation details: + * + * cluster-copier workers pull each partition of each shard of the source cluster and push it to the destination cluster through + * Distributed table (to preform data resharding). So, worker job is a partition of a source shard. + * A job has three states: Active, Finished and Abandoned. Abandoned means that worker died and did not finish the job. + * + * If an error occurred during the copying (a worker failed or a worker did not finish the INSERT), then the whole partition (on + * all destination servers) should be dropped and refilled. So, copying entity is a partition of all destination shards. + * If a failure is detected a special /is_dirty node is created in ZooKeeper signalling that other workers copying the same partition + * should stop, after a refilling procedure should start. + * + * ZooKeeper task node has the following structure: + * /task/path_root - path passed in --task-path parameter + * /description - contains user-defined XML config of the task + * /task_active_workers - contains ephemeral nodes of all currently active workers, used to implement max_workers limitation + * /server_fqdn#PID_timestamp - cluster-copier worker ID + * ... + * /tables - directory with table tasks + * /cluster.db.table1 - directory of table_hits task + * /partition1 - directory for partition1 + * /shards - directory for source cluster shards + * /1 - worker job for the first shard of partition1 of table test.hits + * Contains info about current status (Active or Finished) and worker ID. + * /2 + * ... + * /partition_active_workers + * /1 - for each job in /shards a corresponding ephemeral node created in /partition_active_workers + * It is used to detect Abandoned jobs (if there is Active node in /shards and there is no node in + * /partition_active_workers). + * Also, it is used to track active workers in the partition (when we need to refill the partition we do + * not DROP PARTITION while there are active workers) + * /2 + * ... + * /is_dirty - the node is set if some worker detected that an error occurred (the INSERT is failed or an Abandoned node is + * detected). If the node appeared workers in this partition should stop and start cleaning and refilling + * partition procedure. + * During this procedure a single 'cleaner' worker is selected. The worker waits for stopping all partition + * workers, removes /shards node, executes DROP PARTITION on each destination node and removes /is_dirty node. + * /cleaner- An ephemeral node used to select 'cleaner' worker. Contains ID of the worker. + * /cluster.db.table2 + * ... + */ + +namespace DB +{ + +class ClusterCopierApp : public BaseDaemon +{ +public: + + void initialize(Poco::Util::Application & self) override; + + void handleHelp(const std::string &, const std::string &); + + void defineOptions(Poco::Util::OptionSet & options) override; + + int main(const std::vector &) override; + +private: + + using Base = BaseDaemon; + + void mainImpl(); + + std::string config_xml_path; + std::string task_path; + std::string log_level = "trace"; + bool is_safe_mode = false; + double copy_fault_probability = 0; + bool is_help = false; + + std::string base_dir; + std::string process_path; + std::string process_id; + std::string host_id; +}; + +} diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 5f14604fbf9..6e81f16035c 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -148,12 +148,14 @@ struct TaskStateWithOwner /// Hierarchical description of the tasks +struct ShardPartitionPiece; struct ShardPartition; struct TaskShard; struct TaskTable; struct TaskCluster; struct ClusterPartition; +using PartitionPieces = std::vector; using TasksPartition = std::map>; using ShardInfo = Cluster::ShardInfo; using TaskShardPtr = std::shared_ptr; @@ -162,22 +164,59 @@ using TasksTable = std::list; using ClusterPartitions = std::map>; +struct ShardPartitionPiece +{ + ShardPartitionPiece(ShardPartition & parent, size_t current_piece_number_, bool is_absent_piece_) + : is_absent_piece(is_absent_piece_) + , current_piece_number(current_piece_number_) + , shard_partition(parent) {} + + [[maybe_unused]] String getPartitionPiecePath() const {return "Not implemented.";} + [[maybe_unused]] String getPartitionPieceCleanStartPath() const {return "Not implemented.";} + [[maybe_unused]] String getCommonPartitionPieceIsDirtyPath() const {return "Not implemented.";} + [[maybe_unused]] String getCommonPartitionPieceIsCleanedPath() const {return "Not implemented.";} + + [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const {return "Not implemented.";} + [[maybe_unused]] String getActiveWorkerPath() const {return "Not implemented.";} + + /// On what shards do we have current partition. + [[maybe_unused]] String getPartitionPieceShardsPath() const {return "Not implemented.";} + [[maybe_unused]] String getShardStatusPath() const {return "Not implemented.";} + + bool is_absent_piece; + const size_t current_piece_number; + + ShardPartition & shard_partition; +}; + + /// Just destination partition of a shard /// I don't know what this comment means. /// In short, when we discovered what shards contain currently processing partition, /// This class describes a partition (name) that is stored on the shard (parent). struct ShardPartition { - ShardPartition(TaskShard & parent, String name_quoted_) : task_shard(parent), name(std::move(name_quoted_)) {} + ShardPartition(TaskShard & parent, String name_quoted_, size_t number_of_splits = 10) + : task_shard(parent) + , name(std::move(name_quoted_)) + { pieces.reserve(number_of_splits); } - String getPartitionPath() const; - String getPartitionCleanStartPath() const; - String getCommonPartitionIsDirtyPath() const; - String getCommonPartitionIsCleanedPath() const; - String getPartitionActiveWorkersPath() const; - String getActiveWorkerPath() const; - String getPartitionShardsPath() const; - String getShardStatusPath() const; + /*useful*/ String getPartitionPath() const; + [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + /*useful*/ String getPartitionCleanStartPath() const; + [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + /*useful*/ String getCommonPartitionIsDirtyPath() const; + /*useful*/ String getCommonPartitionIsCleanedPath() const; + /*??????*/ String getPartitionActiveWorkersPath() const; + /*??????*/ String getActiveWorkerPath() const; + /*useful*/ String getPartitionShardsPath() const; + /*useful*/ String getShardStatusPath() const; + + /// What partition pieces are present in current shard. + /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) + /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; + /// Absent pieces have field is_absent_piece equals to true. + PartitionPieces pieces; TaskShard & task_shard; String name; @@ -255,7 +294,7 @@ struct TaskTable TaskCluster & task_cluster; String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const; + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const; String getPartitionIsDirtyPath(const String & partition_name) const; String getPartitionIsCleanedPath(const String & partition_name) const; String getPartitionTaskStatusPath(const String & partition_name) const; @@ -422,8 +461,9 @@ String TaskTable::getPartitionPath(const String & partition_name) const + "/" + escapeForFileName(partition_name); // 201701 } -String TaskTable::getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const +String TaskTable::getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const { + assert(current_piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits } @@ -432,11 +472,23 @@ String ShardPartition::getPartitionCleanStartPath() const return getPartitionPath() + "/clean_start"; } +String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const +{ + assert(current_piece_number < task_shard.task_table.number_of_splits); + return getPartitionPiecePath(current_piece_number) + "/clean_start"; +} + String ShardPartition::getPartitionPath() const { return task_shard.task_table.getPartitionPath(name); } +String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const +{ + assert(current_piece_number < task_shard.task_table.number_of_splits); + return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); +} + String ShardPartition::getShardStatusPath() const { // schema: //tables/
//shards/ diff --git a/dbms/programs/copier/ZookeeperStaff.h b/dbms/programs/copier/ZookeeperStaff.h new file mode 100644 index 00000000000..3133c68933d --- /dev/null +++ b/dbms/programs/copier/ZookeeperStaff.h @@ -0,0 +1,224 @@ +#pragma once + +/** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. + * We assume that we compare values that are not too far away. + * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. + */ +class WrappingUInt32 +{ +public: + UInt32 value; + + explicit WrappingUInt32(UInt32 _value) + : value(_value) + {} + + bool operator<(const WrappingUInt32 & other) const + { + return value != other.value && *this <= other; + } + + bool operator<=(const WrappingUInt32 & other) const + { + const UInt32 HALF = 1 << 31; + return (value <= other.value && other.value - value < HALF) + || (value > other.value && value - other.value > HALF); + } + + bool operator==(const WrappingUInt32 & other) const + { + return value == other.value; + } +}; + +/** Conforming Zxid definition. + * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions + * + * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html + * + * Actually here is the definition of Zxid. + * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). + * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid + * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. + */ +class Zxid +{ +public: + WrappingUInt32 epoch; + WrappingUInt32 counter; + explicit Zxid(UInt64 _zxid) + : epoch(_zxid >> 32) + , counter(_zxid) + {} + + bool operator<=(const Zxid & other) const + { + return (epoch < other.epoch) + || (epoch == other.epoch && counter <= other.counter); + } + + bool operator==(const Zxid & other) const + { + return epoch == other.epoch && counter == other.counter; + } +}; + +/* When multiple ClusterCopiers discover that the target partition is not empty, + * they will attempt to clean up this partition before proceeding to copying. + * + * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established + * based on a happens-before relation between the events. + * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. + * The fact of the partition hygiene is encoded by CleanStateClock. + * + * For you to know what mzxid means: + * + * ZooKeeper Stat Structure: + * The Stat structure for each znode in ZooKeeper is made up of the following fields: + * + * -- czxid + * The zxid of the change that caused this znode to be created. + * + * -- mzxid + * The zxid of the change that last modified this znode. + * + * -- ctime + * The time in milliseconds from epoch when this znode was created. + * + * -- mtime + * The time in milliseconds from epoch when this znode was last modified. + * + * -- version + * The number of changes to the data of this znode. + * + * -- cversion + * The number of changes to the children of this znode. + * + * -- aversion + * The number of changes to the ACL of this znode. + * + * -- ephemeralOwner + * The session id of the owner of this znode if the znode is an ephemeral node. + * If it is not an ephemeral node, it will be zero. + * + * -- dataLength + * The length of the data field of this znode. + * + * -- numChildren + * The number of children of this znode. + * */ + +class LogicalClock +{ +public: + std::optional zxid; + + LogicalClock() = default; + + explicit LogicalClock(UInt64 _zxid) + : zxid(_zxid) + {} + + bool hasHappened() const + { + return bool(zxid); + } + + /// happens-before relation with a reasonable time bound + bool happensBefore(const LogicalClock & other) const + { + return !zxid + || (other.zxid && *zxid <= *other.zxid); + } + + bool operator<=(const LogicalClock & other) const + { + return happensBefore(other); + } + + /// strict equality check + bool operator==(const LogicalClock & other) const + { + return zxid == other.zxid; + } +}; + + +class CleanStateClock +{ +public: + LogicalClock discovery_zxid; + std::optional discovery_version; + + LogicalClock clean_state_zxid; + std::optional clean_state_version; + + std::shared_ptr stale; + + bool is_clean() const + { + return + !is_stale() + && ( + !discovery_zxid.hasHappened() + || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); + } + + bool is_stale() const + { + return stale->load(); + } + + CleanStateClock( + const zkutil::ZooKeeperPtr & zookeeper, + const String & discovery_path, + const String & clean_state_path) + : stale(std::make_shared(false)) + { + Coordination::Stat stat{}; + String _some_data; + auto watch_callback = + [stale = stale] (const Coordination::WatchResponse & rsp) + { + auto logger = &Poco::Logger::get("ClusterCopier"); + if (rsp.error == Coordination::ZOK) + { + switch (rsp.type) + { + case Coordination::CREATED: + LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); + stale->store(true); + break; + case Coordination::CHANGED: + LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); + stale->store(true); + } + } + }; + if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) + { + discovery_zxid = LogicalClock(stat.mzxid); + discovery_version = stat.version; + } + if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) + { + clean_state_zxid = LogicalClock(stat.mzxid); + clean_state_version = stat.version; + } + } + + bool operator==(const CleanStateClock & other) const + { + return !is_stale() + && !other.is_stale() + && discovery_zxid == other.discovery_zxid + && discovery_version == other.discovery_version + && clean_state_zxid == other.clean_state_zxid + && clean_state_version == other.clean_state_version; + } + + bool operator!=(const CleanStateClock & other) const + { + return !(*this == other); + } +}; From b97aa2074deb42a364f64c3abea0ffe334304d82 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Feb 2020 19:15:34 +0300 Subject: [PATCH 004/147] piece observing + config --- dbms/programs/copier/ClusterCopier.h | 22 ++++++++++++++++++++-- dbms/programs/copier/Internals.h | 2 +- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index f4f37b97c26..42519964199 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -162,6 +162,15 @@ public: { task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); task_shard->checked_partitions.emplace(partition_name, true); + + auto shard_partition_it = task_shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + for (int piece_number = 0; piece_number < 10; ++piece_number) + { + bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } } if (!missing_partitions.empty()) @@ -1378,7 +1387,9 @@ public: std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - query += " AND (cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + const size_t number_of_splits = task_table.number_of_splits; + + query += " AND (cityHash64(*) % " + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1393,7 +1404,14 @@ public: Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + if (result != 0) + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + else + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + return result != 0; } /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 6e81f16035c..c2f9509a866 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -608,7 +608,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati name_in_config = table_key; - number_of_splits = config.getUInt64("number_of_splits", 10); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); From 5f1fd17b7c2c62c6d1e9fd9cd82c10c095caff9a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 Feb 2020 13:52:46 +0300 Subject: [PATCH 005/147] ShardPartitionPiece path getters --- dbms/programs/copier/ClusterCopier.h | 22 +- dbms/programs/copier/Internals.h | 447 --------------------- dbms/programs/copier/ShardPartition.h | 95 +++++ dbms/programs/copier/ShardPartitionPiece.h | 83 ++++ dbms/programs/copier/TaskCluster.h | 96 +++++ dbms/programs/copier/TaskTable.h | 243 +++++++++++ 6 files changed, 532 insertions(+), 454 deletions(-) create mode 100644 dbms/programs/copier/ShardPartition.h create mode 100644 dbms/programs/copier/ShardPartitionPiece.h create mode 100644 dbms/programs/copier/TaskCluster.h create mode 100644 dbms/programs/copier/TaskTable.h diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 42519964199..8cc0cf21211 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,6 +1,10 @@ #pragma once #include "Internals.h" +#include "TaskCluster.h" +#include "ShardPartition.h" +#include "ShardPartitionPiece.h" +#include "TaskTable.h" #include "ZookeeperStaff.h" namespace DB @@ -166,7 +170,9 @@ public: auto shard_partition_it = task_shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - for (int piece_number = 0; piece_number < 10; ++piece_number) + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); @@ -472,14 +478,14 @@ public: { CleanStateClock clean_state_clock ( zookeeper, - task_table.getPartitionIsDirtyPath(partition_name), - task_table.getPartitionIsCleanedPath(partition_name) + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) ); Coordination::Stat stat{}; LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) { LOG_INFO(log, "Partition " << partition_name << " become dirty"); @@ -744,8 +750,10 @@ public: /// To save references in the future. auto shard_partition_it = shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - ///FIXME: Remove 10 - for (int piece_number = 0; piece_number < 10; ++piece_number) + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index c2f9509a866..4b5cb4b7a83 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -101,7 +101,6 @@ String getQuotedTable(const DatabaseAndTableName & db_and_table) return getQuotedTable(db_and_table.first, db_and_table.second); } - enum class TaskState { Started = 0, @@ -164,64 +163,6 @@ using TasksTable = std::list; using ClusterPartitions = std::map>; -struct ShardPartitionPiece -{ - ShardPartitionPiece(ShardPartition & parent, size_t current_piece_number_, bool is_absent_piece_) - : is_absent_piece(is_absent_piece_) - , current_piece_number(current_piece_number_) - , shard_partition(parent) {} - - [[maybe_unused]] String getPartitionPiecePath() const {return "Not implemented.";} - [[maybe_unused]] String getPartitionPieceCleanStartPath() const {return "Not implemented.";} - [[maybe_unused]] String getCommonPartitionPieceIsDirtyPath() const {return "Not implemented.";} - [[maybe_unused]] String getCommonPartitionPieceIsCleanedPath() const {return "Not implemented.";} - - [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const {return "Not implemented.";} - [[maybe_unused]] String getActiveWorkerPath() const {return "Not implemented.";} - - /// On what shards do we have current partition. - [[maybe_unused]] String getPartitionPieceShardsPath() const {return "Not implemented.";} - [[maybe_unused]] String getShardStatusPath() const {return "Not implemented.";} - - bool is_absent_piece; - const size_t current_piece_number; - - ShardPartition & shard_partition; -}; - - -/// Just destination partition of a shard -/// I don't know what this comment means. -/// In short, when we discovered what shards contain currently processing partition, -/// This class describes a partition (name) that is stored on the shard (parent). -struct ShardPartition -{ - ShardPartition(TaskShard & parent, String name_quoted_, size_t number_of_splits = 10) - : task_shard(parent) - , name(std::move(name_quoted_)) - { pieces.reserve(number_of_splits); } - - /*useful*/ String getPartitionPath() const; - [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; - /*useful*/ String getPartitionCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; - /*useful*/ String getCommonPartitionIsDirtyPath() const; - /*useful*/ String getCommonPartitionIsCleanedPath() const; - /*??????*/ String getPartitionActiveWorkersPath() const; - /*??????*/ String getActiveWorkerPath() const; - /*useful*/ String getPartitionShardsPath() const; - /*useful*/ String getShardStatusPath() const; - - /// What partition pieces are present in current shard. - /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) - /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; - /// Absent pieces have field is_absent_piece equals to true. - PartitionPieces pieces; - - TaskShard & task_shard; - String name; -}; - struct ShardPriority { @@ -286,129 +227,6 @@ struct ClusterPartition }; -struct TaskTable -{ - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; - - /// Partitions will be splitted into number-of-splits pieces. - /// Each piece will be copied independently. (10 by default) - size_t number_of_splits; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /// A Distributed table definition used to split data - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /// Prioritized list of shards - /// all_shards contains information about all shards in the table. - /// So we have to check whether particular shard have current partiton or not while processing. - TasksShard all_shards; - TasksShard local_shards; - - /// All partitions of the current table. - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String & partition_name) - { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine && random_engine); -}; - - -struct TaskCluster -{ - TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) - : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} - - void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Set (or update) settings and max_workers param - void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Base node for all tasks. Its structure: - /// workers/ - directory with active workers (amount of them is less or equal max_workers) - /// description - node with task configuration - /// table_table1/ - directories with per-partition copying status - String task_zookeeper_path; - - /// Database used to create temporary Distributed tables - String default_local_database; - - /// Limits number of simultaneous workers - UInt64 max_workers = 0; - - /// Base settings for pull and push - Settings settings_common; - /// Settings used to fetch data - Settings settings_pull; - /// Settings used to insert data - Settings settings_push; - - String clusters_prefix; - - /// Subtasks - TasksTable table_tasks; - - std::random_device random_device; - pcg64 random_engine; -}; - - - struct MultiTransactionInfo { int32_t code; @@ -452,107 +270,6 @@ Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) } -/// Path getters - -String TaskTable::getPartitionPath(const String & partition_name) const -{ - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String TaskTable::getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const -{ - assert(current_piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits -} - -String ShardPartition::getPartitionCleanStartPath() const -{ - return getPartitionPath() + "/clean_start"; -} - -String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const -{ - assert(current_piece_number < task_shard.task_table.number_of_splits); - return getPartitionPiecePath(current_piece_number) + "/clean_start"; -} - -String ShardPartition::getPartitionPath() const -{ - return task_shard.task_table.getPartitionPath(name); -} - -String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const -{ - assert(current_piece_number < task_shard.task_table.number_of_splits); - return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); -} - -String ShardPartition::getShardStatusPath() const -{ - // schema: //tables/
//shards/ - // e.g. /root/table_test.hits/201701/shards/1 - return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getPartitionShardsPath() const -{ - return getPartitionPath() + "/shards"; -} - -String ShardPartition::getPartitionActiveWorkersPath() const -{ - return getPartitionPath() + "/partition_active_workers"; -} - -String ShardPartition::getActiveWorkerPath() const -{ - return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getCommonPartitionIsDirtyPath() const -{ - return getPartitionPath() + "/is_dirty"; -} - -String ShardPartition::getCommonPartitionIsCleanedPath() const -{ - return getCommonPartitionIsDirtyPath() + "/cleaned"; -} - -String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const -{ - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/shards"; -} - -String TaskShard::getDescription() const -{ - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String TaskShard::getHostNameExample() const -{ - auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - - static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) { const auto & storage = storage_ast->as(); @@ -600,85 +317,6 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) - : task_cluster(parent) -{ - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) - { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) - { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { - /// Parse sequence of ... - for (const String & key : keys) - { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { @@ -708,91 +346,6 @@ static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, co return res; } -template -void TaskTable::initShards(RandomEngine && random_engine) -{ - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto & shard_info : cluster_pull->getShardsInfo()) - { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) - { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [] (const TaskShardPtr & lhs, UInt8 is_remote) - { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - clusters_prefix = prefix + "remote_servers"; - if (!config.has(clusters_prefix)) - throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); - - Poco::Util::AbstractConfiguration::Keys tables_keys; - config.keys(prefix + "tables", tables_keys); - - for (const auto & table_key : tables_keys) - { - table_tasks.emplace_back(*this, config, prefix + "tables", table_key); - } -} - -void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - max_workers = config.getUInt64(prefix + "max_workers"); - - settings_common = Settings(); - if (config.has(prefix + "settings")) - settings_common.loadSettingsFromConfig(prefix + "settings", config); - - settings_pull = settings_common; - if (config.has(prefix + "settings_pull")) - settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); - - settings_push = settings_common; - if (config.has(prefix + "settings_push")) - settings_push.loadSettingsFromConfig(prefix + "settings_push", config); - - auto set_default_value = [] (auto && setting, auto && default_value) - { - setting = setting.changed ? setting.value : default_value; - }; - - /// Override important settings - settings_pull.readonly = 1; - settings_push.insert_distributed_sync = 1; - set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); - set_default_value(settings_pull.max_threads, 1); - set_default_value(settings_pull.max_block_size, 8192UL); - set_default_value(settings_pull.preferred_block_size_bytes, 0); - set_default_value(settings_push.insert_distributed_timeout, 0); -} - } // end of an anonymous namespace } - diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h new file mode 100644 index 00000000000..2881d3e2f79 --- /dev/null +++ b/dbms/programs/copier/ShardPartition.h @@ -0,0 +1,95 @@ +#pragma once + +#include "Internals.h" +#include "TaskTable.h" + +namespace DB { + +namespace { + + +/// Just destination partition of a shard +/// I don't know what this comment means. +/// In short, when we discovered what shards contain currently processing partition, +/// This class describes a partition (name) that is stored on the shard (parent). +struct ShardPartition { + ShardPartition(TaskShard &parent, String name_quoted_, size_t number_of_splits = 10) + : task_shard(parent), name(std::move(name_quoted_)) { pieces.reserve(number_of_splits); } + + /*useful*/ String getPartitionPath() const; + + [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + + /*useful*/ String getPartitionCleanStartPath() const; + + [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + + /*useful*/ String getCommonPartitionIsDirtyPath() const; + + /*useful*/ String getCommonPartitionIsCleanedPath() const; + + /*??????*/ String getPartitionActiveWorkersPath() const; + + /*??????*/ String getActiveWorkerPath() const; + + /*useful*/ String getPartitionShardsPath() const; + + /*useful*/ String getShardStatusPath() const; + + /// What partition pieces are present in current shard. + /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) + /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; + /// Absent pieces have field is_absent_piece equals to true. + PartitionPieces pieces; + + TaskShard & task_shard; + String name; +}; + +String ShardPartition::getPartitionCleanStartPath() const { + return getPartitionPath() + "/clean_start"; +} + +String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const { + assert(current_piece_number < task_shard.task_table.number_of_splits); + return getPartitionPiecePath(current_piece_number) + "/clean_start"; +} + +String ShardPartition::getPartitionPath() const { + return task_shard.task_table.getPartitionPath(name); +} + +String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const { + assert(current_piece_number < task_shard.task_table.number_of_splits); + return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); +} + +String ShardPartition::getShardStatusPath() const { + // schema: //tables/
//shards/ + // e.g. /root/table_test.hits/201701/shards/1 + return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getPartitionShardsPath() const { + return getPartitionPath() + "/shards"; +} + +String ShardPartition::getPartitionActiveWorkersPath() const { + return getPartitionPath() + "/partition_active_workers"; +} + +String ShardPartition::getActiveWorkerPath() const { + return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getCommonPartitionIsDirtyPath() const { + return getPartitionPath() + "/is_dirty"; +} + +String ShardPartition::getCommonPartitionIsCleanedPath() const { + return getCommonPartitionIsDirtyPath() + "/cleaned"; +} + +} //end of anonymous namespace + +} diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h new file mode 100644 index 00000000000..867a5c4c65a --- /dev/null +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -0,0 +1,83 @@ +#pragma once + +#include "Internals.h" + +namespace DB { + +namespace { + +struct ShardPartitionPiece { + + ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) + : is_absent_piece(is_absent_piece_), current_piece_number(current_piece_number_), + shard_partition(parent) {} + + [[maybe_unused]] String getPartitionPiecePath() const; + + [[maybe_unused]] String getPartitionPieceCleanStartPath() const; + + [[maybe_unused]] String getPartitionPieceIsDirtyPath() const; + + [[maybe_unused]] String getPartitionPieceIsCleanedPath() const; + + [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const; + + [[maybe_unused]] String getActiveWorkerPath() const ; + + /// On what shards do we have current partition. + [[maybe_unused]] String getPartitionPieceShardsPath() const; + + [[maybe_unused]] String getShardStatusPath() const; + + bool is_absent_piece; + const size_t current_piece_number; + + ShardPartition & shard_partition; +}; + + +String ShardPartitionPiece::getPartitionPiecePath() const +{ + return shard_partition.getPartitionPath() + "/piece" + std::to_string(current_piece_number); +} + +String ShardPartitionPiece::getPartitionPieceCleanStartPath() const +{ + return getPartitionPiecePath() + "/clean_start"; +} + +String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const +{ + return getPartitionPiecePath() + "/is_dirty"; +} + +String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const +{ + return getPartitionPieceIsDirtyPath() + "/is_cleaned"; +} + +String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const +{ + return getPartitionPiecePath() + "/partition_active_workers"; +} + +String ShardPartitionPiece::getActiveWorkerPath() const +{ + return getPartitionPieceActiveWorkersPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); +} + +/// On what shards do we have current partition. +String ShardPartitionPiece::getPartitionPieceShardsPath() const +{ + return getPartitionPiecePath() + "/shards"; +} + +String ShardPartitionPiece::getShardStatusPath() const +{ + return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); +} + + +} // end of anonymous namespace + +} diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h new file mode 100644 index 00000000000..8cf67ad78fb --- /dev/null +++ b/dbms/programs/copier/TaskCluster.h @@ -0,0 +1,96 @@ +#pragma once + +#include "Internals.h" + +namespace DB { + +namespace { + +struct TaskCluster { + TaskCluster(const String &task_zookeeper_path_, const String &default_local_database_) + : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} + + void loadTasks(const Poco::Util::AbstractConfiguration &config, const String &base_key = ""); + + /// Set (or update) settings and max_workers param + void reloadSettings(const Poco::Util::AbstractConfiguration &config, const String &base_key = ""); + + /// Base node for all tasks. Its structure: + /// workers/ - directory with active workers (amount of them is less or equal max_workers) + /// description - node with task configuration + /// table_table1/ - directories with per-partition copying status + String task_zookeeper_path; + + /// Database used to create temporary Distributed tables + String default_local_database; + + /// Limits number of simultaneous workers + UInt64 max_workers = 0; + + /// Base settings for pull and push + Settings settings_common; + /// Settings used to fetch data + Settings settings_pull; + /// Settings used to insert data + Settings settings_push; + + String clusters_prefix; + + /// Subtasks + TasksTable table_tasks; + + std::random_device random_device; + pcg64 random_engine; +}; + + +void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration &config, const String &base_key) { + String prefix = base_key.empty() ? "" : base_key + "."; + + clusters_prefix = prefix + "remote_servers"; + if (!config.has(clusters_prefix)) + throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); + + Poco::Util::AbstractConfiguration::Keys tables_keys; + config.keys(prefix + "tables", tables_keys); + + for (const auto &table_key : tables_keys) { + table_tasks.emplace_back(*this, config, prefix + "tables", table_key); + } +} + +void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration &config, const String &base_key) { + String prefix = base_key.empty() ? "" : base_key + "."; + + max_workers = config.getUInt64(prefix + "max_workers"); + + settings_common = Settings(); + if (config.has(prefix + "settings")) + settings_common.loadSettingsFromConfig(prefix + "settings", config); + + settings_pull = settings_common; + if (config.has(prefix + "settings_pull")) + settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); + + settings_push = settings_common; + if (config.has(prefix + "settings_push")) + settings_push.loadSettingsFromConfig(prefix + "settings_push", config); + + auto set_default_value = [](auto &&setting, auto &&default_value) { + setting = setting.changed ? setting.value : default_value; + }; + + /// Override important settings + settings_pull.readonly = 1; + settings_push.insert_distributed_sync = 1; + set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); + set_default_value(settings_pull.max_threads, 1); + set_default_value(settings_pull.max_block_size, 8192UL); + set_default_value(settings_pull.preferred_block_size_bytes, 0); + set_default_value(settings_push.insert_distributed_timeout, 0); +} + +} // end of anonymous namespace + +} + diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h new file mode 100644 index 00000000000..04c8d747ea4 --- /dev/null +++ b/dbms/programs/copier/TaskTable.h @@ -0,0 +1,243 @@ +#pragma once + +#include "Internals.h" +#include "TaskCluster.h" + +namespace DB { + +namespace { + + +struct TaskTable { + TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, + const String &table_key); + + TaskCluster &task_cluster; + + String getPartitionPath(const String &partition_name) const; + + [[maybe_unused]] String + getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const; + + String getCertainPartitionIsDirtyPath(const String &partition_name) const; + + String getCertainPartitionIsCleanedPath(const String &partition_name) const; + + String getCertainPartitionTaskStatusPath(const String &partition_name) const; + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; + + String name_in_config; + + /// Used as task ID + String table_id; + + /// Source cluster and table + String cluster_pull_name; + DatabaseAndTableName table_pull; + + /// Destination cluster and table + String cluster_push_name; + DatabaseAndTableName table_push; + + /// Storage of destination table + String engine_push_str; + ASTPtr engine_push_ast; + ASTPtr engine_push_partition_key_ast; + + /// A Distributed table definition used to split data + String sharding_key_str; + ASTPtr sharding_key_ast; + ASTPtr engine_split_ast; + + /// Additional WHERE expression to filter input data + String where_condition_str; + ASTPtr where_condition_ast; + + /// Resolved clusters + ClusterPtr cluster_pull; + ClusterPtr cluster_push; + + /// Filter partitions that should be copied + bool has_enabled_partitions = false; + Strings enabled_partitions; + NameSet enabled_partitions_set; + + /// Prioritized list of shards + /// all_shards contains information about all shards in the table. + /// So we have to check whether particular shard have current partiton or not while processing. + TasksShard all_shards; + TasksShard local_shards; + + /// All partitions of the current table. + ClusterPartitions cluster_partitions; + NameSet finished_cluster_partitions; + + /// Parition names to process in user-specified order + Strings ordered_partition_names; + + ClusterPartition &getClusterPartition(const String &partition_name) { + auto it = cluster_partitions.find(partition_name); + if (it == cluster_partitions.end()) + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, + ErrorCodes::LOGICAL_ERROR); + return it->second; + } + + Stopwatch watch; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + + template + void initShards(RandomEngine &&random_engine); +}; + + +String TaskTable::getPartitionPath(const String &partition_name) const { + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +String TaskTable::getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const { + assert(current_piece_number < number_of_splits); + return getPartitionPath(partition_name) + "/" + + std::to_string(current_piece_number); // 1...number_of_splits +} + +String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { + return getPartitionPath(partition_name) + "/is_dirty"; +} + +String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { + return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; +} + +String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { + return getPartitionPath(partition_name) + "/shards"; +} + +String TaskShard::getDescription() const { + std::stringstream ss; + ss << "N" << numberInCluster() + << " (having a replica " << getHostNameExample() + << ", pull table " + getQuotedTable(task_table.table_pull) + << " of cluster " + task_table.cluster_pull_name << ")"; + return ss.str(); +} + +String TaskShard::getHostNameExample() const { + auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + return replicas.at(0).readableString(); +} + +TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, + const String &prefix_, + const String &table_key) + : task_cluster(parent) { + String table_prefix = prefix_ + "." + table_key + "."; + + name_in_config = table_key; + + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); + cluster_push_name = config.getString(table_prefix + "cluster_push"); + + table_pull.first = config.getString(table_prefix + "database_pull"); + table_pull.second = config.getString(table_prefix + "table_pull"); + + table_push.first = config.getString(table_prefix + "database_push"); + table_push.second = config.getString(table_prefix + "table_push"); + + /// Used as node name in ZooKeeper + table_id = escapeForFileName(cluster_push_name) + + "." + escapeForFileName(table_push.first) + + "." + escapeForFileName(table_push.second); + + engine_push_str = config.getString(table_prefix + "engine"); + { + ParserStorage parser_storage; + engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + } + + sharding_key_str = config.getString(table_prefix + "sharding_key"); + { + ParserExpressionWithOptionalAlias parser_expression(false); + sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); + engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + sharding_key_ast); + } + + where_condition_str = config.getString(table_prefix + "where_condition", ""); + if (!where_condition_str.empty()) { + ParserExpressionWithOptionalAlias parser_expression(false); + where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); + + // Will use canonical expression form + where_condition_str = queryToString(where_condition_ast); + } + + String enabled_partitions_prefix = table_prefix + "enabled_partitions"; + has_enabled_partitions = config.has(enabled_partitions_prefix); + + if (has_enabled_partitions) { + Strings keys; + config.keys(enabled_partitions_prefix, keys); + + if (keys.empty()) { + /// Parse list of partition from space-separated string + String partitions_str = config.getString(table_prefix + "enabled_partitions"); + boost::trim_if(partitions_str, isWhitespaceASCII); + boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); + } else { + /// Parse sequence of ... + for (const String &key : keys) { + if (!startsWith(key, "partition")) + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, + ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); + } + } + + std::copy(enabled_partitions.begin(), enabled_partitions.end(), + std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + } +} + +template +void TaskTable::initShards(RandomEngine &&random_engine) { + const String &fqdn_name = getFQDNOrHostName(); + std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); + + // Compute the priority + for (auto &shard_info : cluster_pull->getShardsInfo()) { + TaskShardPtr task_shard = std::make_shared(*this, shard_info); + const auto &replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); + task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); + + all_shards.emplace_back(task_shard); + } + + // Sort by priority + std::sort(all_shards.begin(), all_shards.end(), + [](const TaskShardPtr &lhs, const TaskShardPtr &rhs) { + return ShardPriority::greaterPriority(lhs->priority, rhs->priority); + }); + + // Cut local shards + auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, + [](const TaskShardPtr &lhs, UInt8 is_remote) { + return lhs->priority.is_remote < is_remote; + }); + + local_shards.assign(all_shards.begin(), it_first_remote); +} + +} // end of anonymouse namespace + +} From 5795dfa00c7307a3b328c9cfe8e1e67589a7f802 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 Feb 2020 17:23:41 +0300 Subject: [PATCH 006/147] remove anonymous namespaces --- dbms/programs/copier/Internals.h | 5 ----- dbms/programs/copier/ShardPartition.h | 3 --- dbms/programs/copier/ShardPartitionPiece.h | 4 ---- dbms/programs/copier/TaskCluster.h | 2 -- dbms/programs/copier/TaskTable.h | 3 --- 5 files changed, 17 deletions(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4b5cb4b7a83..dfb52dcfc34 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -81,9 +81,6 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -namespace -{ - using DatabaseAndTableName = std::pair; String getQuotedTable(const String & database, const String & table) @@ -346,6 +343,4 @@ static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, co return res; } - -} // end of an anonymous namespace } diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index 2881d3e2f79..37d027eacef 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -5,8 +5,6 @@ namespace DB { -namespace { - /// Just destination partition of a shard /// I don't know what this comment means. @@ -90,6 +88,5 @@ String ShardPartition::getCommonPartitionIsCleanedPath() const { return getCommonPartitionIsDirtyPath() + "/cleaned"; } -} //end of anonymous namespace } diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 867a5c4c65a..28f17a40541 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -4,8 +4,6 @@ namespace DB { -namespace { - struct ShardPartitionPiece { ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) @@ -78,6 +76,4 @@ String ShardPartitionPiece::getShardStatusPath() const } -} // end of anonymous namespace - } diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h index 8cf67ad78fb..3ed7d610d17 100644 --- a/dbms/programs/copier/TaskCluster.h +++ b/dbms/programs/copier/TaskCluster.h @@ -4,7 +4,6 @@ namespace DB { -namespace { struct TaskCluster { TaskCluster(const String &task_zookeeper_path_, const String &default_local_database_) @@ -90,7 +89,6 @@ void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration &config set_default_value(settings_push.insert_distributed_timeout, 0); } -} // end of anonymous namespace } diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h index 04c8d747ea4..c8c292ea3a2 100644 --- a/dbms/programs/copier/TaskTable.h +++ b/dbms/programs/copier/TaskTable.h @@ -5,8 +5,6 @@ namespace DB { -namespace { - struct TaskTable { TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, @@ -238,6 +236,5 @@ void TaskTable::initShards(RandomEngine &&random_engine) { local_shards.assign(all_shards.begin(), it_first_remote); } -} // end of anonymouse namespace } From 93301b8f9f3b07b2d1630ce063fcf2fcbf5f0095 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 14 Feb 2020 22:48:45 +0300 Subject: [PATCH 007/147] Additional of new input format: RegexpRowInputFormat. --- dbms/src/Core/Settings.h | 4 + dbms/src/Formats/FormatFactory.cpp | 5 + dbms/src/Formats/FormatFactory.h | 2 + dbms/src/Formats/FormatSettings.h | 9 + .../Formats/Impl/RegexpRowInputFormat.cpp | 155 ++++++++++++++++++ .../Formats/Impl/RegexpRowInputFormat.h | 43 +++++ .../01079_regexp_input_format.reference | 12 ++ .../0_stateless/01079_regexp_input_format.sh | 27 +++ ...gexp_input_format_skip_unmatched.reference | 3 + ...1080_regexp_input_format_skip_unmatched.sh | 16 ++ 10 files changed, 276 insertions(+) create mode 100644 dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h create mode 100644 dbms/tests/queries/0_stateless/01079_regexp_input_format.reference create mode 100755 dbms/tests/queries/0_stateless/01079_regexp_input_format.sh create mode 100644 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference create mode 100644 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 08c555beb03..13e7f2ae70e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -247,6 +247,10 @@ struct Settings : public SettingsCollection M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ + M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \ + M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ + M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ + \ M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index a8e27054704..c2b890ec631 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -68,6 +68,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings, const Con format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; + format_settings.regexp.regexp = settings.format_regexp; + format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; + format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) @@ -352,10 +355,12 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorAvro(*this); registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); + registerInputFormatProcessorRegexp(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineRegexp(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 7c18971e0eb..7c515dbce90 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -178,6 +178,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory); void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); void registerFileSegmentationEngineCSV(FormatFactory & factory); void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); /// Output only (presentational) formats. @@ -198,5 +199,6 @@ void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory); /// Input only formats. void registerInputFormatProcessorCapnProto(FormatFactory & factory); +void registerInputFormatProcessorRegexp(FormatFactory & factory); } diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index 610768e5d08..1eb95ce2dbf 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -122,6 +122,15 @@ struct FormatSettings Avro avro; + struct Regexp + { + std::string regexp; + std::string escaping_rule; + bool skip_unmatched = false; + }; + + Regexp regexp; + }; } diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp new file mode 100644 index 00000000000..d53c6819c91 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -0,0 +1,155 @@ +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int BAD_ARGUMENTS; +} + +RegexpRowInputFormat::RegexpRowInputFormat( + ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) +{ + field_format = stringToFormat(format_settings_.regexp.escaping_rule); +} + +RegexpRowInputFormat::FieldFormat RegexpRowInputFormat::stringToFormat(const String & format) +{ + if (format == "Escaped") + return FieldFormat::Escaped; + if (format == "Quoted") + return FieldFormat::Quoted; + if (format == "CSV") + return FieldFormat::Csv; + if (format == "JSON") + return FieldFormat::Json; + throw Exception("Unknown field format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); +} + +bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + const auto & type = getPort().getHeader().getByPosition(index).type; + bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); + bool read = true; + ReadBuffer field_buf(matched_fields[index + 1].first, matched_fields[index + 1].length(), 0); + try + { + switch (field_format) + { + case FieldFormat::Escaped: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextEscaped(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextEscaped(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Quoted: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextQuoted(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextQuoted(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Csv: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextCSV(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextCSV(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Json: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextJSON(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); + break; + default: + __builtin_unreachable(); + } + } + catch (Exception & e) + { + throw; + } + return read; +} + +void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) +{ + if (matched_fields.size() != columns.size() + 1) + throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA); + + ext.read_columns.assign(columns.size(), false); + for (size_t columns_index = 0; columns_index < columns.size(); ++columns_index) + { + ext.read_columns[columns_index] = readField(columns_index, columns); + } +} + +bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + if (in.eof()) + return false; + + char * line_end = find_first_symbols<'\n', '\r'>(in.position(), in.buffer().end()); + bool match = std::regex_match(in.position(), line_end, matched_fields, regexp); + + if (!match) + { + if (!format_settings.regexp.skip_unmatched) + throw Exception("Line \"" + std::string(in.position(), line_end) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); + in.position() = line_end + 1; + return true; + } + + readFieldsFromMatch(columns, ext); + + in.position() = line_end + 1; + return true; +} + +void registerInputFormatProcessorRegexp(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("Regexp", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings); + }); +} + +static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + char * pos = in.position(); + bool need_more_data = true; + + while (loadAtPosition(in, memory, pos) && need_more_data) + { + pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + ++pos; + } + + saveUpToPosition(in, memory, pos); + + return loadAtPosition(in, memory, pos); +} + +void registerFileSegmentationEngineRegexp(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h new file mode 100644 index 00000000000..fe920f26fed --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + + +class RegexpRowInputFormat : public IRowInputFormat +{ +public: + RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + + String getName() const override { return "RegexpRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + +private: + enum class FieldFormat + { + Escaped, + Quoted, + Csv, + Json, + }; + + bool readField(size_t index, MutableColumns & columns); + void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); + FieldFormat stringToFormat(const String & format); + + const FormatSettings format_settings; + std::regex regexp; + std::match_results matched_fields; + FieldFormat field_format; +}; + +} diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference b/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference new file mode 100644 index 00000000000..61435bccefc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference @@ -0,0 +1,12 @@ +1 [1,2,3] str1 2020-01-01 +2 [1,2,3] str2 2020-01-02 +3 [1,2,3] str3 2020-01-03 +4 [1,2,3] str4 2020-01-04 +5 [1,2,3] str5 2020-01-05 +6 [1,2,3] str6 2020-01-06 +7 [1,2,3] str7 2020-01-07 +8 [1,2,3] str8 2020-01-08 +9 [1,2,3] str9 2020-01-09 +10 [1,2,3] str10 2020-01-10 +11 [1,2,3] str11 2020-01-11 +12 [1,2,3] str12 2020-01-12 diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh new file mode 100755 index 00000000000..cbaa1bd2162 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, array Array(UInt32), string String, date Date) ENGINE = Memory"; + +echo 'id: 1 array: [1,2,3] string: str1 date: 2020-01-01 +id: 2 array: [1,2,3] string: str2 date: 2020-01-02 +id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped'"; + +echo 'id: 4 array: "[1,2,3]" string: "str4" date: "2020-01-04" +id: 5 array: "[1,2,3]" string: "str5" date: "2020-01-05" +id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV'"; + +echo "id: 7 array: [1,2,3] string: 'str7' date: '2020-01-07' +id: 8 array: [1,2,3] string: 'str8' date: '2020-01-08' +id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted'"; + +echo 'id: 10 array: [1,2,3] string: "str10" date: "2020-01-10" +id: 11 array: [1,2,3] string: "str11" date: "2020-01-11" +id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON'"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp ORDER BY id"; +$CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; + diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference new file mode 100644 index 00000000000..ff57df7316e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference @@ -0,0 +1,3 @@ +1 str1 +2 str2 +4 str4 diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh new file mode 100644 index 00000000000..98bcb0a256a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# . $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGINE = Memory"; + +echo 'id: 1 string: str1 +id: 2 string: str2 +id=3, string=str3 +id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp"; +$CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; + From 038c9a889d5a6ac52582f0461f5f0e50d284295d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Feb 2020 16:26:08 +0300 Subject: [PATCH 008/147] something works --- dbms/programs/copier/ClusterCopier.h | 219 +++++++++++++++++---- dbms/programs/copier/Internals.h | 73 +++++++ dbms/programs/copier/ShardPartitionPiece.h | 2 +- dbms/programs/copier/TaskCluster.h | 2 +- dbms/programs/copier/TaskTable.h | 109 +++++++--- 5 files changed, 342 insertions(+), 63 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 8cc0cf21211..cd0481684dc 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -273,7 +273,8 @@ public: for (TaskTable & task_table : task_cluster->table_tasks) { LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); + << task_table.all_shards.size() << " shards, " + << task_table.local_shards.size() << " of them are local ones"); if (task_table.all_shards.empty()) continue; @@ -507,7 +508,7 @@ public: catch (const Coordination::Exception & e) { LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); + << ". Will recheck the partition. Error: " << e.displayText()); return false; } @@ -525,6 +526,112 @@ public: return true; } + bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + { + bool answer = true; + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) + answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + return answer; + } + + + /* The same as function above + * Assume that we don't know on which shards do we have partition certain piece. + * We'll check them all (I mean shards that contain the whole partition) + * And shards that don't have certain piece MUST mark that piece is_done true. + * */ + bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, + size_t piece_number, const TasksShard & shards_with_partition) + { + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name + << " piece number" + toString(piece_number) + " successfully"); + + auto zookeeper = context.getZooKeeper(); + + /// Collect all shards that contain partition piece number piece_number. + Strings piece_status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; + piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + + } + /// Removes MATERIALIZED and ALIAS columns from create table query static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { @@ -877,6 +984,14 @@ public: Error, }; + + enum class PartititonPieceTaskStatus + { + Active, + Finished, + Error, + }; + /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { @@ -884,7 +999,7 @@ public: try { - res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); + res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); } catch (...) { @@ -905,23 +1020,40 @@ public: return res; } - PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + bool is_unprioritized_task) + { + const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; + + /// ThreadPool maybe ?? + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) + processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + return PartitionTaskStatus::Finished; + + } + + PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) { TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - const size_t number_of_splits = task_table.number_of_splits; - UNUSED(number_of_splits); + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + const size_t number_of_splits = task_table.number_of_splits; + const String primary_key_comma_separated = task_table.primary_key_comma_separated; + UNUSED(number_of_splits); + UNUSED(partition_piece); /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard); + createShardInternalTables(timeouts, task_shard, true, current_piece_number); auto zookeeper = context.getZooKeeper(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - const String current_task_is_active_path = task_partition.getActiveWorkerPath(); - const String current_task_status_path = task_partition.getShardStatusPath(); + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); /// Auxiliary functions: @@ -935,25 +1067,27 @@ public: else if (clock.discovery_version) { LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); + zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); } else { LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); } }; /// Returns SELECT query filtering current partition and applying user filter auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false, size_t current_piece_number = 0) + bool enable_splitting = false) { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) - query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; if (!limit.empty()) @@ -964,11 +1098,11 @@ public: }; /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); - LOG_DEBUG(log, "Processing " << current_task_status_path); + LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); LogicalClock task_start_clock; { @@ -982,7 +1116,7 @@ public: && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_status_path); + zookeeper->createAncestors(current_task_piece_status_path); } else { @@ -1001,17 +1135,17 @@ public: } /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_is_active_path); + zookeeper->createAncestors(current_task_piece_is_active_path); zkutil::EphemeralNodeHolderPtr partition_task_node_holder; try { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); } catch (const Coordination::Exception & e) { if (e.code == Coordination::ZNODEEXISTS) { - LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); + LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); return PartitionTaskStatus::Active; } @@ -1022,17 +1156,17 @@ public: /// create blocking node to signal cleaning up if it is abandoned { String status_data; - if (zookeeper->tryGet(current_task_status_path, status_data)) + if (zookeeper->tryGet(current_task_piece_status_path, status_data)) { TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); return PartitionTaskStatus::Finished; } // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); @@ -1064,7 +1198,7 @@ public: if (count != 0) { Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); + zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning if (stat_shards.numChildren == 0) @@ -1085,7 +1219,7 @@ public: /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1097,13 +1231,18 @@ public: create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } - zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); + zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); } /// Try create table (if not exists) on each shard { + /// Define push table for current partition piece + auto database_and_table_for_current_piece= std::pair( + task_table.table_push.first, + task_table.table_push.second + ".piece_" + toString(current_piece_number)); + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - task_table.table_push, task_table.engine_push_ast); + database_and_table_for_current_piece, task_table.engine_push_ast); create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); @@ -1173,7 +1312,7 @@ public: throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); + future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); /// check_period_milliseconds should less than average insert time of single block /// Otherwise, the insertion will slow a little bit @@ -1222,7 +1361,7 @@ public: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1234,7 +1373,7 @@ public: create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } - zookeeper->set(current_task_status_path, state_finished, 0); + zookeeper->set(current_task_piece_status_path, state_finished, 0); } LOG_INFO(log, "Partition " << task_partition.name << " copied"); @@ -1283,7 +1422,8 @@ public: return parseQuery(parser_create_query, create_query_pull_str, 0); } - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) + /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) { TaskTable & task_table = task_shard.task_table; @@ -1295,7 +1435,8 @@ public: String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName( + working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; @@ -1303,16 +1444,16 @@ public: context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_split_ast = task_table.engine_split_ast; + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); + auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); dropAndCreateLocalTable(create_table_pull_ast); if (create_split) - dropAndCreateLocalTable(create_table_split_ast); + dropAndCreateLocalTable(create_table_split_piece_ast); } @@ -1396,8 +1537,10 @@ public: + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; const size_t number_of_splits = task_table.number_of_splits; + const String & primary_key_comma_separated = task_table.primary_key_comma_separated; - query += " AND (cityHash64(*) % " + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; + query += " AND (cityHash64(" + primary_key_comma_separated + ") % " + + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index dfb52dcfc34..d4f565b63df 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -313,6 +313,79 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } +/* + * Choosing a Primary Key that Differs from the Sorting Key + * It is possible to specify a primary key (an expression with values that are written in the index file for each mark) + * that is different from the sorting key (an expression for sorting the rows in data parts). + * In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. + * This feature is helpful when using the SummingMergeTree and AggregatingMergeTree table engines. + * In a common case when using these engines, the table has two types of columns: dimensions and measures. + * Typical queries aggregate values of measure columns with arbitrary GROUP BY and filtering by dimensions. + * Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, + * it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns + * and this list must be frequently updated with newly added dimensions. + * In this case it makes sense to leave only a few columns in the primary key that will provide efficient + * range scans and add the remaining dimension columns to the sorting key tuple. + * ALTER of the sorting key is a lightweight operation because when a new column is simultaneously added t + * o the table and to the sorting key, existing data parts don't need to be changed. + * Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, + * the data is sorted by both the old and new sorting keys at the moment of table modification. + * + * */ +[[maybe_unused]] static ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + /// FIXME + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.primary_key) + return storage.primary_key->clone(); + + return storage.order_by->clone(); +} + +[[maybe_unused]] static String createCommaSeparatedStringFrom(const Strings & strings) +{ + String answer; + for (auto & string: strings) + answer += string + ", "; + + /// Remove last comma and space + answer.pop_back(); + answer.pop_back(); + return answer; +} + +[[maybe_unused]] static Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +{ + const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + + ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); + ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + + Strings answer; + answer.reserve(primary_key_or_order_by_arguments.size()); + + for (auto & column : primary_key_or_order_by_arguments) + answer.push_back(column->getColumnName()); + + return answer; +} + static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 28f17a40541..303407d1d5b 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -36,7 +36,7 @@ struct ShardPartitionPiece { String ShardPartitionPiece::getPartitionPiecePath() const { - return shard_partition.getPartitionPath() + "/piece" + std::to_string(current_piece_number); + return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); } String ShardPartitionPiece::getPartitionPieceCleanStartPath() const diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h index 3ed7d610d17..0df6f7e6a6b 100644 --- a/dbms/programs/copier/TaskCluster.h +++ b/dbms/programs/copier/TaskCluster.h @@ -53,7 +53,7 @@ void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration &config, con Poco::Util::AbstractConfiguration::Keys tables_keys; config.keys(prefix + "tables", tables_keys); - for (const auto &table_key : tables_keys) { + for (const auto & table_key : tables_keys) { table_tasks.emplace_back(*this, config, prefix + "tables", table_key); } } diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h index c8c292ea3a2..10aaf9334a6 100644 --- a/dbms/programs/copier/TaskTable.h +++ b/dbms/programs/copier/TaskTable.h @@ -2,26 +2,50 @@ #include "Internals.h" #include "TaskCluster.h" +#include "ext/range.h" namespace DB { struct TaskTable { - TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, - const String &table_key); + TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, + const String & table_key); - TaskCluster &task_cluster; + TaskCluster & task_cluster; - String getPartitionPath(const String &partition_name) const; + /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() + /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. - [[maybe_unused]] String - getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const; + String getPartitionPath(const String & partition_name) const; - String getCertainPartitionIsDirtyPath(const String &partition_name) const; + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; - String getCertainPartitionIsCleanedPath(const String &partition_name) const; + String getCertainPartitionIsDirtyPath(const String & partition_name) const; - String getCertainPartitionTaskStatusPath(const String &partition_name) const; + [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not Implemented"; + } + + String getCertainPartitionIsCleanedPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + String getCertainPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -32,6 +56,9 @@ struct TaskTable { /// Used as task ID String table_id; + /// Column names in primary key + String primary_key_comma_separated; + /// Source cluster and table String cluster_pull_name; DatabaseAndTableName table_pull; @@ -41,14 +68,37 @@ struct TaskTable { DatabaseAndTableName table_push; /// Storage of destination table + /// (tables that are stored on each shard of target cluster) String engine_push_str; ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; - /// A Distributed table definition used to split data + /* + * A Distributed table definition used to split data + * Distributed table will be created on each shard of default + * cluster to perform data copying and resharding + * */ String sharding_key_str; ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; + ASTPtr main_engine_split_ast; + + + /* + * Auxuliary table engines used to perform partition piece copying. + * Each AST represent table engine for certatin piece number. + * After copying partition piece is Ok, this piece will be moved to the main + * target table. All this tables are stored on each shard as the main table. + * We have to use separate tables for partition pieces because of the atomicity of copying. + * Also if we want to move some partition to another table, the partition keys have to be the same. + * */ + + + /* + * To copy partiton piece form one cluster to another we have to use Distributed table. + * In case of usage separate table (engine_push) for each partiton piece, + * we have to use many Distributed tables. + * */ + ASTs auxiliary_engine_split_asts; /// Additional WHERE expression to filter input data String where_condition_str; @@ -63,9 +113,11 @@ struct TaskTable { Strings enabled_partitions; NameSet enabled_partitions_set; - /// Prioritized list of shards - /// all_shards contains information about all shards in the table. - /// So we have to check whether particular shard have current partiton or not while processing. + /** + * Prioritized list of shards + * all_shards contains information about all shards in the table. + * So we have to check whether particular shard have current partiton or not while processing. + */ TasksShard all_shards; TasksShard local_shards; @@ -76,7 +128,7 @@ struct TaskTable { /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition &getClusterPartition(const String &partition_name) { + ClusterPartition & getClusterPartition(const String &partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) throw Exception("There are no cluster partition " + partition_name + " in " + table_id, @@ -99,10 +151,10 @@ String TaskTable::getPartitionPath(const String &partition_name) const { + "/" + escapeForFileName(partition_name); // 201701 } -String TaskTable::getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const { - assert(current_piece_number < number_of_splits); +String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { + assert(piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + - std::to_string(current_piece_number); // 1...number_of_splits + std::to_string(piece_number); // 1...number_of_splits } String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { @@ -131,9 +183,8 @@ String TaskShard::getHostNameExample() const { return replicas.at(0).readableString(); } -TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, - const String &prefix_, - const String &table_key) +TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, + const String & prefix_, const String & table_key) : task_cluster(parent) { String table_prefix = prefix_ + "." + table_key + "."; @@ -160,14 +211,26 @@ TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguratio ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); + + auxiliary_engine_split_asts.reserve(number_of_splits); { ParserExpressionWithOptionalAlias parser_expression(false); sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + + for (const auto piece_number : ext::range(0, number_of_splits)) + { + auxiliary_engine_split_asts.emplace_back + ( + createASTStorageDistributed(cluster_push_name, table_push.first, + table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + ); + } } where_condition_str = config.getString(table_prefix + "where_condition", ""); @@ -209,7 +272,7 @@ TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguratio template void TaskTable::initShards(RandomEngine &&random_engine) { - const String &fqdn_name = getFQDNOrHostName(); + const String & fqdn_name = getFQDNOrHostName(); std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); // Compute the priority From 6527df83a1997d97e73b477c3320c467e8cc5be6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Feb 2020 16:39:22 +0300 Subject: [PATCH 009/147] copier split --- dbms/programs/copier/ClusterCopier.cpp | 1615 ++++++++++++++++++++++++ dbms/programs/copier/ClusterCopier.h | 1573 +---------------------- 2 files changed, 1649 insertions(+), 1539 deletions(-) create mode 100644 dbms/programs/copier/ClusterCopier.cpp diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp new file mode 100644 index 00000000000..684bfddf7d4 --- /dev/null +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -0,0 +1,1615 @@ +#include "ClusterCopier.h" + +namespace DB +{ + +void ClusterCopier::init() +{ + auto zookeeper = context.getZooKeeper(); + + task_description_watch_callback = [this] (const Coordination::WatchResponse & response) + { + if (response.error != Coordination::ZOK) + return; + UInt64 version = ++task_descprtion_version; + LOG_DEBUG(log, "Task description should be updated, local version " << version); + }; + + task_description_path = task_zookeeper_path + "/description"; + task_cluster = std::make_unique(task_zookeeper_path, working_database_name); + + reloadTaskDescription(); + task_cluster_initial_config = task_cluster_current_config; + + task_cluster->loadTasks(*task_cluster_initial_config); + context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + + /// Set up shards and their priority + task_cluster->random_engine.seed(task_cluster->random_device()); + for (auto & task_table : task_cluster->table_tasks) + { + task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + task_table.cluster_push = context.getCluster(task_table.cluster_push_name); + task_table.initShards(task_cluster->random_engine); + } + + LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); + + /// Do not initialize tables, will make deferred initialization in process() + + zookeeper->createAncestors(getWorkersPathVersion() + "/"); + zookeeper->createAncestors(getWorkersPath() + "/"); +} + +template +decltype(auto) ClusterCopier::retry(T && func, UInt64 max_tries = 100) +{ + std::exception_ptr exception; + + for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) + { + try + { + return func(); + } + catch (...) + { + exception = std::current_exception(); + if (try_number < max_tries) + { + tryLogCurrentException(log, "Will retry"); + std::this_thread::sleep_for(default_sleep_time); + } + } + } + + std::rethrow_exception(exception); +} + + +void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) +{ + TaskTable & task_table = task_shard->task_table; + + LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); + + auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; + auto existing_partitions_names = retry(get_partitions, 60); + Strings filtered_partitions_names; + Strings missing_partitions; + + /// Check that user specified correct partition names + auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) + { + MutableColumnPtr column_dummy = type->createColumn(); + ReadBufferFromString rb(partition_text_quoted); + + try + { + type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); + } + catch (Exception & e) + { + throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); + } + }; + + if (task_table.has_enabled_partitions) + { + /// Process partition in order specified by + for (const String & partition_name : task_table.enabled_partitions) + { + /// Check that user specified correct partition names + check_partition_format(task_shard->partition_key_column.type, partition_name); + + auto it = existing_partitions_names.find(partition_name); + + /// Do not process partition if it is not in enabled_partitions list + if (it == existing_partitions_names.end()) + { + missing_partitions.emplace_back(partition_name); + continue; + } + + filtered_partitions_names.emplace_back(*it); + } + + for (const String & partition_name : existing_partitions_names) + { + if (!task_table.enabled_partitions_set.count(partition_name)) + { + LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " + << "enabled_partitions of " << task_table.table_id); + } + } + } + else + { + for (const String & partition_name : existing_partitions_names) + filtered_partitions_names.emplace_back(partition_name); + } + + for (const String & partition_name : filtered_partitions_names) + { + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + task_shard->checked_partitions.emplace(partition_name, true); + + auto shard_partition_it = task_shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) + { + bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + + if (!missing_partitions.empty()) + { + std::stringstream ss; + for (const String & missing_partition : missing_partitions) + ss << " " << missing_partition; + + LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " + << task_shard->getDescription() << " :" << ss.str()); + } + + LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); +} + +/// Compute set of partitions, assume set of partitions aren't changed during the processing +void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) +{ + /// Fetch partitions list from a shard + { + ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + + for (const TaskShardPtr & task_shard : task_table.all_shards) + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + + LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); + thread_pool.wait(); + } +} + +void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) +{ + auto local_task_description_path = task_path + "/description"; + + String task_config_str; + { + ReadBufferFromFile in(task_file); + readStringUntilEOF(task_config_str, in); + } + if (task_config_str.empty()) + return; + + auto zookeeper = context.getZooKeeper(); + + zookeeper->createAncestors(local_task_description_path); + auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + if (code && force) + zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + + LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); +} + +void ClusterCopier::reloadTaskDescription() +{ + auto zookeeper = context.getZooKeeper(); + task_description_watch_zookeeper = zookeeper; + + String task_config_str; + Coordination::Stat stat{}; + int code; + + zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); + if (code) + throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); + + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); + auto config = getConfigurationFromXMLString(task_config_str); + + /// Setup settings + task_cluster->reloadSettings(*config); + context.getSettingsRef() = task_cluster->settings_common; + + task_cluster_current_config = config; + task_description_current_stat = stat; +} + +void ClusterCopier::updateConfigIfNeeded() +{ + UInt64 version_to_update = task_descprtion_version; + bool is_outdated_version = task_descprtion_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + + if (!is_outdated_version && !is_expired_session) + return; + + LOG_DEBUG(log, "Updating task description"); + reloadTaskDescription(); + + task_descprtion_current_version = version_to_update; +} + +void ClusterCopier::process(const ConnectionTimeouts & timeouts) +{ + for (TaskTable & task_table : task_cluster->table_tasks) + { + LOG_INFO(log, "Process table task " << task_table.table_id << " with " + << task_table.all_shards.size() << " shards, " + << task_table.local_shards.size() << " of them are local ones"); + + if (task_table.all_shards.empty()) + continue; + + /// Discover partitions of each shard and total set of partitions + if (!task_table.has_enabled_partitions) + { + /// If there are no specified enabled_partitions, we must discover them manually + discoverTablePartitions(timeouts, task_table); + + /// After partitions of each shard are initialized, initialize cluster partitions + for (const TaskShardPtr & task_shard : task_table.all_shards) + { + for (const auto & partition_elem : task_shard->partition_tasks) + { + const String & partition_name = partition_elem.first; + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + } + } + + for (auto & partition_elem : task_table.cluster_partitions) + { + const String & partition_name = partition_elem.first; + + for (const TaskShardPtr & task_shard : task_table.all_shards) + task_shard->checked_partitions.emplace(partition_name); + + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + else + { + /// If enabled_partitions are specified, assume that each shard has all partitions + /// We will refine partition set of each shard in future + + for (const String & partition_name : task_table.enabled_partitions) + { + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + + task_table.watch.restart(); + + /// Retry table processing + bool table_is_done = false; + for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) + { + if (tryProcessTable(timeouts, task_table)) + { + table_is_done = true; + break; + } + } + + if (!table_is_done) + { + throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", + ErrorCodes::UNFINISHED); + } + } +} + +/// Disables DROP PARTITION commands that used to clear data after errors +void ClusterCopier::setSafeMode(bool is_safe_mode_ = true) +{ + is_safe_mode = is_safe_mode_; +} + +void ClusterCopier::setCopyFaultProbability(double copy_fault_probability_) +{ + copy_fault_probability = copy_fault_probability_; +} + +/// Protected section + +String ClusterCopier::getWorkersPath() const +{ + return task_cluster->task_zookeeper_path + "/task_active_workers"; +} + +String ClusterCopier::getWorkersPathVersion() const +{ + return getWorkersPath() + "_version"; +} + +String ClusterCopier::getCurrentWorkerNodePath() const +{ + return getWorkersPath() + "/" + host_id; +} + +zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNeed( + const zkutil::ZooKeeperPtr & zookeeper, + const String & description, + bool unprioritized) +{ + std::chrono::milliseconds current_sleep_time = default_sleep_time; + static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec + + if (unprioritized) + std::this_thread::sleep_for(current_sleep_time); + + String workers_version_path = getWorkersPathVersion(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); + + UInt64 num_bad_version_errors = 0; + + while (true) + { + updateConfigIfNeeded(); + + Coordination::Stat stat{}; + zookeeper->get(workers_version_path, &stat); + auto version = stat.version; + zookeeper->get(workers_path, &stat); + + if (static_cast(stat.numChildren) >= task_cluster->max_workers) + { + LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" + << ". Postpone processing " << description); + + if (unprioritized) + current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); + + std::this_thread::sleep_for(current_sleep_time); + num_bad_version_errors = 0; + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); + ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + + if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) + return std::make_shared(current_worker_path, *zookeeper, false, false, description); + + if (code == Coordination::ZBADVERSION) + { + ++num_bad_version_errors; + + /// Try to make fast retries + if (num_bad_version_errors > 3) + { + LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); + std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); + std::this_thread::sleep_for(random_sleep_time); + num_bad_version_errors = 0; + } + } + else + throw Coordination::Exception(code); + } + } +} + +/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. + * State of some task could change during the processing. + * We have to ensure that all shards have the finished state and there is no dirty flag. + * Moreover, we have to check status twice and check zxid, because state can change during the checking. + */ +bool ClusterCopier::checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) +{ + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + + auto zookeeper = context.getZooKeeper(); + + Strings status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name + << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; +} + +bool ClusterCopier::checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) +{ + bool answer = true; + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) + answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + return answer; +} + + +/* The same as function above + * Assume that we don't know on which shards do we have partition certain piece. + * We'll check them all (I mean shards that contain the whole partition) + * And shards that don't have certain piece MUST mark that piece is_done true. + * */ +bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, + size_t piece_number, const TasksShard & shards_with_partition) +{ + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name + << " piece number" + toString(piece_number) + " successfully"); + + auto zookeeper = context.getZooKeeper(); + + /// Collect all shards that contain partition piece number piece_number. + Strings piece_status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; + piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + +} + +/// Removes MATERIALIZED and ALIAS columns from create table query +static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) +{ + const ASTs & column_asts = query_ast->as().columns_list->columns->children; + auto new_columns = std::make_shared(); + + for (const ASTPtr & column_ast : column_asts) + { + const auto & column = column_ast->as(); + + if (!column.default_specifier.empty()) + { + ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); + if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) + continue; + } + + new_columns->children.emplace_back(column_ast->clone()); + } + + ASTPtr new_query_ast = query_ast->clone(); + auto & new_query = new_query_ast->as(); + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, new_columns); + if (auto indices = query_ast->as()->columns_list->indices) + new_columns_list->set(new_columns_list->indices, indices->clone()); + + new_query.replace(new_query.columns_list, new_columns_list); + + return new_query_ast; +} + +/// Replaces ENGINE and table name in a create query +static std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +{ + const auto & create = create_query_ast->as(); + auto res = std::make_shared(create); + + if (create.storage == nullptr || new_storage_ast == nullptr) + throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); + + res->database = new_table.first; + res->table = new_table.second; + + res->children.clear(); + res->set(res->columns_list, create.columns_list->clone()); + res->set(res->storage, new_storage_ast->clone()); + + return res; +} + +bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) +{ + if (is_safe_mode) + throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); + + TaskTable & task_table = task_partition.task_shard.task_table; + + const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + + zkutil::EphemeralNodeHolder::Ptr cleaner_holder; + try + { + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + + throw; + } + + Coordination::Stat stat{}; + if (zookeeper->exists(current_partition_active_workers_dir, &stat)) + { + if (stat.numChildren != 0) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + else + { + zookeeper->remove(current_partition_active_workers_dir); + } + } + + { + zkutil::EphemeralNodeHolder::Ptr active_workers_lock; + try + { + active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); + return false; + } + + throw; + } + + // Lock the dirty flag + zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); + zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + /// Remove all status nodes + { + Strings children; + if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) + for (const auto & child : children) + { + zookeeper->removeRecursive(current_shards_path + "/" + child); + } + } + + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + query += " DROP PARTITION " + task_partition.name + ""; + + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + + ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + /// It is important, DROP PARTITION must be done synchronously + settings_push.replication_alter_partitions_sync = 2; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// Limit number of max executing replicas to 1 + UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + + if (num_shards < cluster_push->getShardCount()) + { + LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); + return false; + } + + /// Update the locking node + if (!my_clock.is_stale()) + { + zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); + if (my_clock.clean_state_version) + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); + else + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); + } + else + { + LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + /// clean state is stale + return false; + } + + LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) + zookeeper->set(current_shards_path, host_id); + } + + LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + return true; +} + + +static constexpr UInt64 max_table_tries = 1000; +static constexpr UInt64 max_shard_partition_tries = 600; + +bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) +{ + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + bool previous_shard_is_instantly_finished = false; + + /// Process each partition that is present in cluster + for (const String & partition_name : task_table.ordered_partition_names) + { + if (!task_table.cluster_partitions.count(partition_name)) + throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); + + ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; + + Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. + TasksShard expected_shards; + UInt64 num_failed_shards = 0; + + ++cluster_partition.total_tries; + + LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); + + /// Process each source shard having current partition and copy current partition + /// NOTE: shards are sorted by "distance" to current host + bool has_shard_to_process = false; + for (const TaskShardPtr & shard : task_table.all_shards) + { + /// Does shard have a node with current partition? + if (shard->partition_tasks.count(partition_name) == 0) + { + /// If not, did we check existence of that partition previously? + if (shard->checked_partitions.count(partition_name) == 0) + { + auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; + bool has_partition = retry(check_shard_has_partition); + + shard->checked_partitions.emplace(partition_name); + + if (has_partition) + { + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); + /// To save references in the future. + auto shard_partition_it = shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) + { + auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + else + { + LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); + continue; + } + } + else + { + /// We have already checked that partition, but did not discover it + previous_shard_is_instantly_finished = true; + continue; + } + } + + auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. + if (it_shard_partition == shard->partition_tasks.end()) + throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); + auto & partition = it_shard_partition->second; + + expected_shards.emplace_back(shard); + + /// Do not sleep if there is a sequence of already processed shards to increase startup + bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; + PartitionTaskStatus task_status = PartitionTaskStatus::Error; + bool was_error = false; + has_shard_to_process = true; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); + + /// Exit if success + if (task_status == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (task_status == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + if (task_status == PartitionTaskStatus::Error) + ++num_failed_shards; + + previous_shard_is_instantly_finished = !was_error; + } + + cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); + + /// Check that whole cluster partition is done + /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done + bool partition_is_done = num_failed_shards == 0; + try + { + partition_is_done = + !has_shard_to_process + || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + } + catch (...) + { + tryLogCurrentException(log); + partition_is_done = false; + } + + if (partition_is_done) + { + task_table.finished_cluster_partitions.emplace(partition_name); + + task_table.bytes_copied += cluster_partition.bytes_copied; + task_table.rows_copied += cluster_partition.rows_copied; + double elapsed = cluster_partition.elapsed_time_seconds; + + LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name + << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" + << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" + << " and " << cluster_partition.blocks_copied << " source blocks are copied"); + + if (cluster_partition.rows_copied) + { + LOG_INFO(log, "Average partition speed: " + << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); + } + + if (task_table.rows_copied) + { + LOG_INFO(log, "Average table " << task_table.table_id << " speed: " + << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); + } + } + } + + UInt64 required_partitions = task_table.cluster_partitions.size(); + UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); + bool table_is_done = finished_partitions >= required_partitions; + + if (!table_is_done) + { + LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." + << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); + } + + return table_is_done; +} + +/// Job for copying partition from particular shard. +PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) +{ + PartitionTaskStatus res; + + try + { + res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); + res = PartitionTaskStatus::Error; + } + + /// At the end of each task check if the config is updated + try + { + updateConfigIfNeeded(); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while updating the config"); + } + + return res; +} + +PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + bool is_unprioritized_task) +{ + const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; + + /// ThreadPool maybe ?? + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) + processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + return PartitionTaskStatus::Finished; + +} + +PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) +{ + TaskShard & task_shard = task_partition.task_shard; + TaskTable & task_table = task_shard.task_table; + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + + const size_t number_of_splits = task_table.number_of_splits; + const String primary_key_comma_separated = task_table.primary_key_comma_separated; + UNUSED(number_of_splits); + UNUSED(partition_piece); + /// We need to update table definitions for each partition, it could be changed after ALTER + createShardInternalTables(timeouts, task_shard, true, current_piece_number); + + auto zookeeper = context.getZooKeeper(); + + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + + /// Auxiliary functions: + + /// Creates is_dirty node to initialize DROP PARTITION + auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + { + if (clock.is_stale()) + LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + else if (!clock.is_clean()) + LOG_DEBUG(log, "Thank you, Captain Obvious"); + else if (clock.discovery_version) + { + LOG_DEBUG(log, "Updating clean state clock"); + zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); + } + else + { + LOG_DEBUG(log, "Creating clean state clock"); + zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + } + }; + + /// Returns SELECT query filtering current partition and applying user filter + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false) + { + String query; + query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); + /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) + query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + + if (enable_splitting) + query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + if (!limit.empty()) + query += " LIMIT " + limit; + + ParserQuery p_query(query.data() + query.size()); + return parseQuery(p_query, query, 0); + }; + + /// Load balancing + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); + + LOG_DEBUG(log, "Processing " << current_task_piece_status_path); + + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + LogicalClock task_start_clock; + { + Coordination::Stat stat{}; + if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + } + + /// Do not start if partition is dirty, try to clean it + if (clean_state_clock.is_clean() + && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + zookeeper->createAncestors(current_task_piece_status_path); + } + else + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + + try + { + tryDropPartition(task_partition, zookeeper, clean_state_clock); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred when clean partition"); + } + + return PartitionTaskStatus::Error; + } + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_task_piece_is_active_path); + zkutil::EphemeralNodeHolderPtr partition_task_node_holder; + try + { + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); + return PartitionTaskStatus::Active; + } + + throw; + } + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_task_piece_status_path, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); + return PartitionTaskStatus::Finished; + } + + // Task is abandoned, initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + + /// Check that destination partition is empty if we are first worker + /// NOTE: this check is incorrect if pull and push tables have different partition key! + String clean_start_status; + if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + { + zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); + // Maybe we are the first worker + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + UInt64 count; + { + Context local_context = context; + // Use pull (i.e. readonly) settings, but fetch data from destination servers + local_context.getSettingsRef() = task_cluster->settings_pull; + local_context.getSettingsRef().skip_unavailable_shards = true; + + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; + } + + if (count != 0) + { + Coordination::Stat stat_shards{}; + zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); + + /// NOTE: partition is still fresh if dirt discovery happens before cleaning + if (stat_shards.numChildren == 0) + { + LOG_WARNING(log, "There are no workers for partition " << task_partition.name + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + } + /// At this point, we need to sync that the destination table is clean + /// before any actual work + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); + } + + /// Try create table (if not exists) on each shard + { + /// Define push table for current partition piece + auto database_and_table_for_current_piece= std::pair( + task_table.table_push.first, + task_table.table_push.second + ".piece_" + toString(current_piece_number)); + + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + database_and_table_for_current_piece, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Do the copying + { + bool inject_fault = false; + if (copy_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < copy_fault_probability; + } + + // Select all fields + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + + LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() + << " : " << queryToString(query_select_ast)); + + ASTPtr query_insert_ast; + { + String query; + query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + + ParserQuery p_query(query.data() + query.size()); + query_insert_ast = parseQuery(p_query, query, 0); + + LOG_DEBUG(log, "Executing INSERT query: " << query); + } + + try + { + /// Custom INSERT SELECT implementation + Context context_select = context; + context_select.getSettingsRef() = task_cluster->settings_pull; + + Context context_insert = context; + context_insert.getSettingsRef() = task_cluster->settings_push; + + BlockInputStreamPtr input; + BlockOutputStreamPtr output; + { + BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); + BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); + + input = io_select.in; + output = io_insert.out; + } + + /// Fail-fast optimization to abort copying when the current clean state expires + std::future future_is_dirty_checker; + + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + constexpr UInt64 check_period_milliseconds = 500; + + /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data + auto cancel_check = [&] () + { + if (zookeeper->expired()) + throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + + if (!future_is_dirty_checker.valid()) + future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); + + /// check_period_milliseconds should less than average insert time of single block + /// Otherwise, the insertion will slow a little bit + if (watch.elapsedMilliseconds() >= check_period_milliseconds) + { + Coordination::ExistsResponse status = future_is_dirty_checker.get(); + + if (status.error != Coordination::ZNONODE) + { + LogicalClock dirt_discovery_epoch (status.stat.mzxid); + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) + return false; + throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + } + } + + return false; + }; + + /// Update statistics + /// It is quite rough: bytes_copied don't take into account DROP PARTITION. + auto update_stats = [&cluster_partition] (const Block & block) + { + cluster_partition.bytes_copied += block.bytes(); + cluster_partition.rows_copied += block.rows(); + cluster_partition.blocks_copied += 1; + }; + + /// Main work is here + copyData(*input, *output, cancel_check, update_stats); + + // Just in case + if (future_is_dirty_checker.valid()) + future_is_dirty_checker.get(); + + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + return PartitionTaskStatus::Error; + } + } + + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->set(current_task_piece_status_path, state_finished, 0); + } + + LOG_INFO(log, "Partition " << task_partition.name << " copied"); + return PartitionTaskStatus::Finished; +} + +void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast) +{ + const auto & create = create_ast->as(); + dropLocalTableIfExists({create.database, create.table}); + + InterpreterCreateQuery interpreter(create_ast, context); + interpreter.execute(); +} + +void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_name) const +{ + auto drop_ast = std::make_shared(); + drop_ast->if_exists = true; + drop_ast->database = table_name.first; + drop_ast->table = table_name.second; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); +} + +String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) +{ + String query = "SHOW CREATE TABLE " + getQuotedTable(table); + Block block = getBlockWithAllStreamData(std::make_shared( + connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); + + return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); +} + +ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) +{ + /// Fetch and parse (possibly) new definition + auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); + String create_query_pull_str = getRemoteCreateTable( + task_shard.task_table.table_pull, + *connection_entry, + &task_cluster->settings_pull); + + ParserCreateQuery parser_create_query; + return parseQuery(parser_create_query, create_query_pull_str, 0); +} + +/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. +void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) +{ + TaskTable & task_table = task_shard.task_table; + + /// We need to update table definitions for each part, it could be changed after ALTER + task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + + /// Create local Distributed tables: + /// a table fetching data from current shard and a table inserting data to the whole destination cluster + String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; + String split_shard_prefix = ".split."; + task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName( + working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); + + /// Create special cluster with single shard + String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; + ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); + context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); + + auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; + + auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); + auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); + + dropAndCreateLocalTable(create_table_pull_ast); + + if (create_split) + dropAndCreateLocalTable(create_table_split_piece_ast); +} + + +std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + String query; + { + WriteBufferFromOwnString wb; + wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" + << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; + query = wb.str(); + } + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + + std::set res; + if (block) + { + ColumnWithTypeAndName & column = block.getByPosition(0); + task_shard.partition_key_column = column; + + for (size_t i = 0; i < column.column->size(); ++i) + { + WriteBufferFromOwnString wb; + column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); + res.emplace(wb.str()); + } + } + + LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); + + return res; +} + +bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; +} + +/// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); +/// Just copypaste the function above +bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + const size_t number_of_splits = task_table.number_of_splits; + const String & primary_key_comma_separated = task_table.primary_key_comma_separated; + + query += " AND (cityHash64(" + primary_key_comma_separated + ") % " + + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + if (result != 0) + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + else + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + return result != 0; +} + +/** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster + * Returns number of shards for which at least one replica executed query successfully + */ +UInt64 ClusterCopier::executeQueryOnCluster( + const ClusterPtr & cluster, + const String & query, + const ASTPtr & query_ast_ = nullptr, + const Settings * settings = nullptr, + PoolMode pool_mode = PoolMode::GET_ALL, + UInt64 max_successful_executions_per_shard = 0) const +{ + auto num_shards = cluster->getShardsInfo().size(); + std::vector per_shard_num_successful_replicas(num_shards, 0); + + ASTPtr query_ast; + if (query_ast_ == nullptr) + { + ParserQuery p_query(query.data() + query.size()); + query_ast = parseQuery(p_query, query, 0); + } + else + query_ast = query_ast_; + + + /// We need to execute query on one replica at least + auto do_for_shard = [&] (UInt64 shard_index) + { + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); + UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); + num_successful_executions = 0; + + auto increment_and_check_exit = [&] () -> bool + { + ++num_successful_executions; + return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; + }; + + UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + UInt64 num_local_replicas = shard.getLocalNodeCount(); + UInt64 num_remote_replicas = num_replicas - num_local_replicas; + + /// In that case we don't have local replicas, but do it just in case + for (UInt64 i = 0; i < num_local_replicas; ++i) + { + auto interpreter = InterpreterFactory::get(query_ast, context); + interpreter->execute(); + + if (increment_and_check_exit()) + return; + } + + /// Will try to make as many as possible queries + if (shard.hasRemoteConnections()) + { + Settings current_settings = settings ? *settings : task_cluster->settings_common; + current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + + for (auto & connection : connections) + { + if (connection.isNull()) + continue; + + try + { + /// CREATE TABLE and DROP PARTITION queries return empty block + RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + NullBlockOutputStream output{Block{}}; + copyData(stream, output); + + if (increment_and_check_exit()) + return; + } + catch (const Exception &) + { + LOG_INFO(log, getCurrentExceptionMessage(false, true)); + } + } + } + }; + + { + ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); + + for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) + thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + + thread_pool.wait(); + } + + UInt64 successful_shards = 0; + for (UInt64 num_replicas : per_shard_num_successful_replicas) + successful_shards += (num_replicas > 0); + + return successful_shards; +} +} diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index cd0481684dc..5c438427ee7 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -36,504 +36,50 @@ public: { } - void init() - { - auto zookeeper = context.getZooKeeper(); - - task_description_watch_callback = [this] (const Coordination::WatchResponse & response) - { - if (response.error != Coordination::ZOK) - return; - UInt64 version = ++task_descprtion_version; - LOG_DEBUG(log, "Task description should be updated, local version " << version); - }; - - task_description_path = task_zookeeper_path + "/description"; - task_cluster = std::make_unique(task_zookeeper_path, working_database_name); - - reloadTaskDescription(); - task_cluster_initial_config = task_cluster_current_config; - - task_cluster->loadTasks(*task_cluster_initial_config); - context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); - - /// Set up shards and their priority - task_cluster->random_engine.seed(task_cluster->random_device()); - for (auto & task_table : task_cluster->table_tasks) - { - task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - task_table.cluster_push = context.getCluster(task_table.cluster_push_name); - task_table.initShards(task_cluster->random_engine); - } - - LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); - - /// Do not initialize tables, will make deferred initialization in process() - - zookeeper->createAncestors(getWorkersPathVersion() + "/"); - zookeeper->createAncestors(getWorkersPath() + "/"); - } + void init(); template - decltype(auto) retry(T && func, UInt64 max_tries = 100) - { - std::exception_ptr exception; + decltype(auto) retry(T && func, UInt64 max_tries = 100); - for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) - { - try - { - return func(); - } - catch (...) - { - exception = std::current_exception(); - if (try_number < max_tries) - { - tryLogCurrentException(log, "Will retry"); - std::this_thread::sleep_for(default_sleep_time); - } - } - } - - std::rethrow_exception(exception); - } - - - void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) - { - TaskTable & task_table = task_shard->task_table; - - LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); - - auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; - auto existing_partitions_names = retry(get_partitions, 60); - Strings filtered_partitions_names; - Strings missing_partitions; - - /// Check that user specified correct partition names - auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) - { - MutableColumnPtr column_dummy = type->createColumn(); - ReadBufferFromString rb(partition_text_quoted); - - try - { - type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); - } - catch (Exception & e) - { - throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); - } - }; - - if (task_table.has_enabled_partitions) - { - /// Process partition in order specified by - for (const String & partition_name : task_table.enabled_partitions) - { - /// Check that user specified correct partition names - check_partition_format(task_shard->partition_key_column.type, partition_name); - - auto it = existing_partitions_names.find(partition_name); - - /// Do not process partition if it is not in enabled_partitions list - if (it == existing_partitions_names.end()) - { - missing_partitions.emplace_back(partition_name); - continue; - } - - filtered_partitions_names.emplace_back(*it); - } - - for (const String & partition_name : existing_partitions_names) - { - if (!task_table.enabled_partitions_set.count(partition_name)) - { - LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " - << "enabled_partitions of " << task_table.table_id); - } - } - } - else - { - for (const String & partition_name : existing_partitions_names) - filtered_partitions_names.emplace_back(partition_name); - } - - for (const String & partition_name : filtered_partitions_names) - { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); - task_shard->checked_partitions.emplace(partition_name, true); - - auto shard_partition_it = task_shard->partition_tasks.find(partition_name); - PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - - const size_t number_of_splits = task_table.number_of_splits; - - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) - { - bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); - shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); - } - } - - if (!missing_partitions.empty()) - { - std::stringstream ss; - for (const String & missing_partition : missing_partitions) - ss << " " << missing_partition; - - LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " - << task_shard->getDescription() << " :" << ss.str()); - } - - LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); - } + void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard); /// Compute set of partitions, assume set of partitions aren't changed during the processing - void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) - { - /// Fetch partitions list from a shard - { - ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0); - for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force); - LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); - thread_pool.wait(); - } - } + void reloadTaskDescription(); - void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) - { - auto local_task_description_path = task_path + "/description"; + void updateConfigIfNeeded(); - String task_config_str; - { - ReadBufferFromFile in(task_file); - readStringUntilEOF(task_config_str, in); - } - if (task_config_str.empty()) - return; - - auto zookeeper = context.getZooKeeper(); - - zookeeper->createAncestors(local_task_description_path); - auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - if (code && force) - zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - - LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); - } - - void reloadTaskDescription() - { - auto zookeeper = context.getZooKeeper(); - task_description_watch_zookeeper = zookeeper; - - String task_config_str; - Coordination::Stat stat{}; - int code; - - zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code) - throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - - LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); - auto config = getConfigurationFromXMLString(task_config_str); - - /// Setup settings - task_cluster->reloadSettings(*config); - context.getSettingsRef() = task_cluster->settings_common; - - task_cluster_current_config = config; - task_description_current_stat = stat; - } - - void updateConfigIfNeeded() - { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); - - if (!is_outdated_version && !is_expired_session) - return; - - LOG_DEBUG(log, "Updating task description"); - reloadTaskDescription(); - - task_descprtion_current_version = version_to_update; - } - - void process(const ConnectionTimeouts & timeouts) - { - for (TaskTable & task_table : task_cluster->table_tasks) - { - LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " - << task_table.local_shards.size() << " of them are local ones"); - - if (task_table.all_shards.empty()) - continue; - - /// Discover partitions of each shard and total set of partitions - if (!task_table.has_enabled_partitions) - { - /// If there are no specified enabled_partitions, we must discover them manually - discoverTablePartitions(timeouts, task_table); - - /// After partitions of each shard are initialized, initialize cluster partitions - for (const TaskShardPtr & task_shard : task_table.all_shards) - { - for (const auto & partition_elem : task_shard->partition_tasks) - { - const String & partition_name = partition_elem.first; - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - } - } - - for (auto & partition_elem : task_table.cluster_partitions) - { - const String & partition_name = partition_elem.first; - - for (const TaskShardPtr & task_shard : task_table.all_shards) - task_shard->checked_partitions.emplace(partition_name); - - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - else - { - /// If enabled_partitions are specified, assume that each shard has all partitions - /// We will refine partition set of each shard in future - - for (const String & partition_name : task_table.enabled_partitions) - { - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - - task_table.watch.restart(); - - /// Retry table processing - bool table_is_done = false; - for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) - { - if (tryProcessTable(timeouts, task_table)) - { - table_is_done = true; - break; - } - } - - if (!table_is_done) - { - throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", - ErrorCodes::UNFINISHED); - } - } - } + void process(const ConnectionTimeouts & timeouts); /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true) - { - is_safe_mode = is_safe_mode_; - } - - void setCopyFaultProbability(double copy_fault_probability_) - { - copy_fault_probability = copy_fault_probability_; - } + void setSafeMode(bool is_safe_mode_ = true); + void setCopyFaultProbability(double copy_fault_probability_); protected: - String getWorkersPath() const - { - return task_cluster->task_zookeeper_path + "/task_active_workers"; - } + String getWorkersPath() const; - String getWorkersPathVersion() const - { - return getWorkersPath() + "_version"; - } + String getWorkersPathVersion() const; - String getCurrentWorkerNodePath() const - { - return getWorkersPath() + "/" + host_id; - } + String getCurrentWorkerNodePath() const; zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, const String & description, - bool unprioritized) - { - std::chrono::milliseconds current_sleep_time = default_sleep_time; - static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec - - if (unprioritized) - std::this_thread::sleep_for(current_sleep_time); - - String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); - - UInt64 num_bad_version_errors = 0; - - while (true) - { - updateConfigIfNeeded(); - - Coordination::Stat stat{}; - zookeeper->get(workers_version_path, &stat); - auto version = stat.version; - zookeeper->get(workers_path, &stat); - - if (static_cast(stat.numChildren) >= task_cluster->max_workers) - { - LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" - << ". Postpone processing " << description); - - if (unprioritized) - current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); - - std::this_thread::sleep_for(current_sleep_time); - num_bad_version_errors = 0; - } - else - { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); - ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); - Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); - - if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); - - if (code == Coordination::ZBADVERSION) - { - ++num_bad_version_errors; - - /// Try to make fast retries - if (num_bad_version_errors > 3) - { - LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); - std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); - std::this_thread::sleep_for(random_sleep_time); - num_bad_version_errors = 0; - } - } - else - throw Coordination::Exception(code); - } - } - } + bool unprioritized); /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. * State of some task could change during the processing. * We have to ensure that all shards have the finished state and there is no dirty flag. * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); - - auto zookeeper = context.getZooKeeper(); - - Strings status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - } - - bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - bool answer = true; - for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) - answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); - return answer; - } + bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); + bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); /* The same as function above * Assume that we don't know on which shards do we have partition certain piece. @@ -541,440 +87,21 @@ public: * And shards that don't have certain piece MUST mark that piece is_done true. * */ bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, - size_t piece_number, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name - << " piece number" + toString(piece_number) + " successfully"); - - auto zookeeper = context.getZooKeeper(); - - /// Collect all shards that contain partition piece number piece_number. - Strings piece_status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; - piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : piece_status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : piece_status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " - << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - - } + size_t piece_number, const TasksShard & shards_with_partition); /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) - { - const ASTs & column_asts = query_ast->as().columns_list->columns->children; - auto new_columns = std::make_shared(); - - for (const ASTPtr & column_ast : column_asts) - { - const auto & column = column_ast->as(); - - if (!column.default_specifier.empty()) - { - ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); - if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) - continue; - } - - new_columns->children.emplace_back(column_ast->clone()); - } - - ASTPtr new_query_ast = query_ast->clone(); - auto & new_query = new_query_ast->as(); - - auto new_columns_list = std::make_shared(); - new_columns_list->set(new_columns_list->columns, new_columns); - if (auto indices = query_ast->as()->columns_list->indices) - new_columns_list->set(new_columns_list->indices, indices->clone()); - - new_query.replace(new_query.columns_list, new_columns_list); - - return new_query_ast; - } + static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) - { - const auto & create = create_query_ast->as(); - auto res = std::make_shared(create); - - if (create.storage == nullptr || new_storage_ast == nullptr) - throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); - - res->database = new_table.first; - res->table = new_table.second; - - res->children.clear(); - res->set(res->columns_list, create.columns_list->clone()); - res->set(res->storage, new_storage_ast->clone()); - - return res; - } - - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) - { - if (is_safe_mode) - throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); - - TaskTable & task_table = task_partition.task_shard.task_table; - - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - - zkutil::EphemeralNodeHolder::Ptr cleaner_holder; - try - { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - - throw; - } - - Coordination::Stat stat{}; - if (zookeeper->exists(current_partition_active_workers_dir, &stat)) - { - if (stat.numChildren != 0) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - else - { - zookeeper->remove(current_partition_active_workers_dir); - } - } - - { - zkutil::EphemeralNodeHolder::Ptr active_workers_lock; - try - { - active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); - return false; - } - - throw; - } - - // Lock the dirty flag - zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - /// Remove all status nodes - { - Strings children; - if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) - for (const auto & child : children) - { - zookeeper->removeRecursive(current_shards_path + "/" + child); - } - } - - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); - query += " DROP PARTITION " + task_partition.name + ""; - - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - - ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - /// It is important, DROP PARTITION must be done synchronously - settings_push.replication_alter_partitions_sync = 2; - - LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); - - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } - - /// Update the locking node - if (!my_clock.is_stale()) - { - zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); - if (my_clock.clean_state_version) - zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); - else - zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); - } - else - { - LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); - /// clean state is stale - return false; - } - - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); - if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) - zookeeper->set(current_shards_path, host_id); - } - - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); - return true; - } + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, + const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); + bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) - { - /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint - bool previous_shard_is_instantly_finished = false; - - /// Process each partition that is present in cluster - for (const String & partition_name : task_table.ordered_partition_names) - { - if (!task_table.cluster_partitions.count(partition_name)) - throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); - - ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; - - Stopwatch watch; - /// We will check all the shards of the table and check if they contain current partition. - TasksShard expected_shards; - UInt64 num_failed_shards = 0; - - ++cluster_partition.total_tries; - - LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); - - /// Process each source shard having current partition and copy current partition - /// NOTE: shards are sorted by "distance" to current host - bool has_shard_to_process = false; - for (const TaskShardPtr & shard : task_table.all_shards) - { - /// Does shard have a node with current partition? - if (shard->partition_tasks.count(partition_name) == 0) - { - /// If not, did we check existence of that partition previously? - if (shard->checked_partitions.count(partition_name) == 0) - { - auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; - bool has_partition = retry(check_shard_has_partition); - - shard->checked_partitions.emplace(partition_name); - - if (has_partition) - { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); - LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); - /// To save references in the future. - auto shard_partition_it = shard->partition_tasks.find(partition_name); - PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - - const size_t number_of_splits = task_table.number_of_splits; - - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) - { - auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); - shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); - } - } - else - { - LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); - continue; - } - } - else - { - /// We have already checked that partition, but did not discover it - previous_shard_is_instantly_finished = true; - continue; - } - } - - auto it_shard_partition = shard->partition_tasks.find(partition_name); - /// Previously when we discovered that shard does not contain current partition, we skipped it. - /// At this moment partition have to be present. - if (it_shard_partition == shard->partition_tasks.end()) - throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); - auto & partition = it_shard_partition->second; - - expected_shards.emplace_back(shard); - - /// Do not sleep if there is a sequence of already processed shards to increase startup - bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; - bool was_error = false; - has_shard_to_process = true; - for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) - { - task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); - - /// Exit if success - if (task_status == PartitionTaskStatus::Finished) - break; - - was_error = true; - - /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) - break; - - /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); - } - - if (task_status == PartitionTaskStatus::Error) - ++num_failed_shards; - - previous_shard_is_instantly_finished = !was_error; - } - - cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); - - /// Check that whole cluster partition is done - /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; - try - { - partition_is_done = - !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); - } - catch (...) - { - tryLogCurrentException(log); - partition_is_done = false; - } - - if (partition_is_done) - { - task_table.finished_cluster_partitions.emplace(partition_name); - - task_table.bytes_copied += cluster_partition.bytes_copied; - task_table.rows_copied += cluster_partition.rows_copied; - double elapsed = cluster_partition.elapsed_time_seconds; - - LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name - << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" - << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" - << " and " << cluster_partition.blocks_copied << " source blocks are copied"); - - if (cluster_partition.rows_copied) - { - LOG_INFO(log, "Average partition speed: " - << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); - } - - if (task_table.rows_copied) - { - LOG_INFO(log, "Average table " << task_table.table_id << " speed: " - << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); - } - } - } - - UInt64 required_partitions = task_table.cluster_partitions.size(); - UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); - bool table_is_done = finished_partitions >= required_partitions; - - if (!table_is_done) - { - LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." - << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); - } - - return table_is_done; - } - + bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Execution status of a task enum class PartitionTaskStatus @@ -993,577 +120,33 @@ public: }; /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - PartitionTaskStatus res; - - try - { - res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; - } - - /// At the end of each task check if the config is updated - try - { - updateConfigIfNeeded(); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while updating the config"); - } - - return res; - } + PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - bool is_unprioritized_task) - { - const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - - /// ThreadPool maybe ?? - for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) - processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - - return PartitionTaskStatus::Finished; - - } + bool is_unprioritized_task); PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task) - { - TaskShard & task_shard = task_partition.task_shard; - TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + const size_t current_piece_number, bool is_unprioritized_task); - const size_t number_of_splits = task_table.number_of_splits; - const String primary_key_comma_separated = task_table.primary_key_comma_separated; - UNUSED(number_of_splits); - UNUSED(partition_piece); - /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard, true, current_piece_number); + void dropAndCreateLocalTable(const ASTPtr & create_ast); - auto zookeeper = context.getZooKeeper(); + void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const; - const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); - const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); - const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); - const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); - /// Auxiliary functions: - - /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) - { - if (clock.is_stale()) - LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); - else if (!clock.is_clean()) - LOG_DEBUG(log, "Thank you, Captain Obvious"); - else if (clock.discovery_version) - { - LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); - } - else - { - LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); - } - }; - - /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false) - { - String query; - query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); - /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) - query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - - if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - if (!limit.empty()) - query += " LIMIT " + limit; - - ParserQuery p_query(query.data() + query.size()); - return parseQuery(p_query, query, 0); - }; - - /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); - - LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - - CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - - LogicalClock task_start_clock; - { - Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - } - - /// Do not start if partition is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_piece_status_path); - } - else - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); - - try - { - tryDropPartition(task_partition, zookeeper, clean_state_clock); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred when clean partition"); - } - - return PartitionTaskStatus::Error; - } - - /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_piece_is_active_path); - zkutil::EphemeralNodeHolderPtr partition_task_node_holder; - try - { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); - return PartitionTaskStatus::Active; - } - - throw; - } - - /// Exit if task has been already processed; - /// create blocking node to signal cleaning up if it is abandoned - { - String status_data; - if (zookeeper->tryGet(current_task_piece_status_path, status_data)) - { - TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); - if (status.state == TaskState::Finished) - { - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; - } - - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - - /// Check that destination partition is empty if we are first worker - /// NOTE: this check is incorrect if pull and push tables have different partition key! - String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") - { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", - *zookeeper, host_id); - // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); - UInt64 count; - { - Context local_context = context; - // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.getSettingsRef() = task_cluster->settings_pull; - local_context.getSettingsRef().skip_unavailable_shards = true; - - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); - count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; - } - - if (count != 0) - { - Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); - - /// NOTE: partition is still fresh if dirt discovery happens before cleaning - if (stat_shards.numChildren == 0) - { - LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); - } - /// At this point, we need to sync that the destination table is clean - /// before any actual work - - /// Try start processing, create node about it - { - String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); - } - - /// Try create table (if not exists) on each shard - { - /// Define push table for current partition piece - auto database_and_table_for_current_piece= std::pair( - task_table.table_push.first, - task_table.table_push.second + ".piece_" + toString(current_piece_number)); - - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - database_and_table_for_current_piece, task_table.engine_push_ast); - create_query_push_ast->as().if_not_exists = true; - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); - } - - /// Do the copying - { - bool inject_fault = false; - if (copy_fault_probability > 0) - { - double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); - inject_fault = value < copy_fault_probability; - } - - // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); - - LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() - << " : " << queryToString(query_select_ast)); - - ASTPtr query_insert_ast; - { - String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; - - ParserQuery p_query(query.data() + query.size()); - query_insert_ast = parseQuery(p_query, query, 0); - - LOG_DEBUG(log, "Executing INSERT query: " << query); - } - - try - { - /// Custom INSERT SELECT implementation - Context context_select = context; - context_select.getSettingsRef() = task_cluster->settings_pull; - - Context context_insert = context; - context_insert.getSettingsRef() = task_cluster->settings_push; - - BlockInputStreamPtr input; - BlockOutputStreamPtr output; - { - BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - - input = io_select.in; - output = io_insert.out; - } - - /// Fail-fast optimization to abort copying when the current clean state expires - std::future future_is_dirty_checker; - - Stopwatch watch(CLOCK_MONOTONIC_COARSE); - constexpr UInt64 check_period_milliseconds = 500; - - /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data - auto cancel_check = [&] () - { - if (zookeeper->expired()) - throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - - if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); - - /// check_period_milliseconds should less than average insert time of single block - /// Otherwise, the insertion will slow a little bit - if (watch.elapsedMilliseconds() >= check_period_milliseconds) - { - Coordination::ExistsResponse status = future_is_dirty_checker.get(); - - if (status.error != Coordination::ZNONODE) - { - LogicalClock dirt_discovery_epoch (status.stat.mzxid); - if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) - return false; - throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - } - } - - return false; - }; - - /// Update statistics - /// It is quite rough: bytes_copied don't take into account DROP PARTITION. - auto update_stats = [&cluster_partition] (const Block & block) - { - cluster_partition.bytes_copied += block.bytes(); - cluster_partition.rows_copied += block.rows(); - cluster_partition.blocks_copied += 1; - }; - - /// Main work is here - copyData(*input, *output, cancel_check, update_stats); - - // Just in case - if (future_is_dirty_checker.valid()) - future_is_dirty_checker.get(); - - if (inject_fault) - throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); - return PartitionTaskStatus::Error; - } - } - - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) - { - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->set(current_task_piece_status_path, state_finished, 0); - } - - LOG_INFO(log, "Partition " << task_partition.name << " copied"); - return PartitionTaskStatus::Finished; - } - - void dropAndCreateLocalTable(const ASTPtr & create_ast) - { - const auto & create = create_ast->as(); - dropLocalTableIfExists({create.database, create.table}); - - InterpreterCreateQuery interpreter(create_ast, context); - interpreter.execute(); - } - - void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const - { - auto drop_ast = std::make_shared(); - drop_ast->if_exists = true; - drop_ast->database = table_name.first; - drop_ast->table = table_name.second; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - - String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) - { - String query = "SHOW CREATE TABLE " + getQuotedTable(table); - Block block = getBlockWithAllStreamData(std::make_shared( - connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); - - return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); - } - - ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - /// Fetch and parse (possibly) new definition - auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); - String create_query_pull_str = getRemoteCreateTable( - task_shard.task_table.table_pull, - *connection_entry, - &task_cluster->settings_pull); - - ParserCreateQuery parser_create_query; - return parseQuery(parser_create_query, create_query_pull_str, 0); - } + ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) - { - TaskTable & task_table = task_shard.task_table; + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0); - /// We need to update table definitions for each part, it could be changed after ALTER - task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); - /// Create local Distributed tables: - /// a table fetching data from current shard and a table inserting data to the whole destination cluster - String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; - String split_shard_prefix = ".split."; - task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); - - /// Create special cluster with single shard - String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; - ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); - context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); - - auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; - - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); - - dropAndCreateLocalTable(create_table_pull_ast); - - if (create_split) - dropAndCreateLocalTable(create_table_split_piece_ast); - } - - - std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" - << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; - query = wb.str(); - } - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); - - std::set res; - if (block) - { - ColumnWithTypeAndName & column = block.getByPosition(0); - task_shard.partition_key_column = column; - - for (size_t i = 0; i < column.column->size(); ++i) - { - WriteBufferFromOwnString wb; - column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); - res.emplace(wb.str()); - } - } - - LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); - - return res; - } - - bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; - } + bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name); /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); /// Just copypaste the function above bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - const size_t number_of_splits = task_table.number_of_splits; - const String & primary_key_comma_separated = task_table.primary_key_comma_separated; - - query += " AND (cityHash64(" + primary_key_comma_separated + ") % " - + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); - if (result != 0) - LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); - else - LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); - return result != 0; - } + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully @@ -1574,95 +157,7 @@ public: const ASTPtr & query_ast_ = nullptr, const Settings * settings = nullptr, PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const - { - auto num_shards = cluster->getShardsInfo().size(); - std::vector per_shard_num_successful_replicas(num_shards, 0); - - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, 0); - } - else - query_ast = query_ast_; - - - /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) - { - const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); - UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); - num_successful_executions = 0; - - auto increment_and_check_exit = [&] () -> bool - { - ++num_successful_executions; - return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; - }; - - UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - UInt64 num_local_replicas = shard.getLocalNodeCount(); - UInt64 num_remote_replicas = num_replicas - num_local_replicas; - - /// In that case we don't have local replicas, but do it just in case - for (UInt64 i = 0; i < num_local_replicas; ++i) - { - auto interpreter = InterpreterFactory::get(query_ast, context); - interpreter->execute(); - - if (increment_and_check_exit()) - return; - } - - /// Will try to make as many as possible queries - if (shard.hasRemoteConnections()) - { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); - - for (auto & connection : connections) - { - if (connection.isNull()) - continue; - - try - { - /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; - NullBlockOutputStream output{Block{}}; - copyData(stream, output); - - if (increment_and_check_exit()) - return; - } - catch (const Exception &) - { - LOG_INFO(log, getCurrentExceptionMessage(false, true)); - } - } - } - }; - - { - ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); - - for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); - - thread_pool.wait(); - } - - UInt64 successful_shards = 0; - for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); - - return successful_shards; - } + UInt64 max_successful_executions_per_shard = 0) const; private: String task_zookeeper_path; From 70e8e1db26b13800324ce001f2798e4e7afbf47d Mon Sep 17 00:00:00 2001 From: Avogar Date: Tue, 18 Feb 2020 16:53:12 +0300 Subject: [PATCH 010/147] Fix style errors and tests --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 +- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h | 2 +- dbms/tests/queries/0_stateless/01079_regexp_input_format.sh | 2 +- .../0_stateless/01080_regexp_input_format_skip_unmatched.sh | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index d53c6819c91..f0f3d10755b 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes } RegexpRowInputFormat::RegexpRowInputFormat( - ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) + ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) { field_format = stringToFormat(format_settings_.regexp.escaping_rule); diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index fe920f26fed..48a711d0cde 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -15,7 +15,7 @@ class ReadBuffer; class RegexpRowInputFormat : public IRowInputFormat { public: - RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "RegexpRowInputFormat"; } diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh index cbaa1bd2162..81ccf35e678 100755 --- a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh @@ -1,4 +1,4 @@ -#!/usr/bin/env bash#!/usr/bin/env bash +#!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh old mode 100644 new mode 100755 index 98bcb0a256a..f46a6239795 --- a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash -# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# . $CURDIR/../shell_config.sh +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGINE = Memory"; From b08db134c1b3bec91b869ced70daeea4f371d6f0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 12:00:41 +0300 Subject: [PATCH 011/147] merging --- dbms/programs/copier/TaskTable.h | 303 -------------------------- dbms/programs/copier/ZookeeperStaff.h | 224 ------------------- 2 files changed, 527 deletions(-) delete mode 100644 dbms/programs/copier/TaskTable.h delete mode 100644 dbms/programs/copier/ZookeeperStaff.h diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h deleted file mode 100644 index 10aaf9334a6..00000000000 --- a/dbms/programs/copier/TaskTable.h +++ /dev/null @@ -1,303 +0,0 @@ -#pragma once - -#include "Internals.h" -#include "TaskCluster.h" -#include "ext/range.h" - -namespace DB { - - -struct TaskTable { - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() - /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. - - String getPartitionPath(const String & partition_name) const; - - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; - - String getCertainPartitionIsDirtyPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not Implemented"; - } - - String getCertainPartitionIsCleanedPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } - - String getCertainPartitionTaskStatusPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } - - /// Partitions will be splitted into number-of-splits pieces. - /// Each piece will be copied independently. (10 by default) - size_t number_of_splits; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Column names in primary key - String primary_key_comma_separated; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - /// (tables that are stored on each shard of target cluster) - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /* - * A Distributed table definition used to split data - * Distributed table will be created on each shard of default - * cluster to perform data copying and resharding - * */ - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr main_engine_split_ast; - - - /* - * Auxuliary table engines used to perform partition piece copying. - * Each AST represent table engine for certatin piece number. - * After copying partition piece is Ok, this piece will be moved to the main - * target table. All this tables are stored on each shard as the main table. - * We have to use separate tables for partition pieces because of the atomicity of copying. - * Also if we want to move some partition to another table, the partition keys have to be the same. - * */ - - - /* - * To copy partiton piece form one cluster to another we have to use Distributed table. - * In case of usage separate table (engine_push) for each partiton piece, - * we have to use many Distributed tables. - * */ - ASTs auxiliary_engine_split_asts; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /** - * Prioritized list of shards - * all_shards contains information about all shards in the table. - * So we have to check whether particular shard have current partiton or not while processing. - */ - TasksShard all_shards; - TasksShard local_shards; - - /// All partitions of the current table. - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String &partition_name) { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, - ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine &&random_engine); -}; - - -String TaskTable::getPartitionPath(const String &partition_name) const { - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { - assert(piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + - std::to_string(piece_number); // 1...number_of_splits -} - -String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { - return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { - return getPartitionPath(partition_name) + "/shards"; -} - -String TaskShard::getDescription() const { - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String TaskShard::getHostNameExample() const { - auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, - const String & prefix_, const String & table_key) - : task_cluster(parent) { - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - - auxiliary_engine_split_asts.reserve(number_of_splits); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, - sharding_key_ast); - - for (const auto piece_number : ext::range(0, number_of_splits)) - { - auxiliary_engine_split_asts.emplace_back - ( - createASTStorageDistributed(cluster_push_name, table_push.first, - table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) - ); - } - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } else { - /// Parse sequence of ... - for (const String &key : keys) { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), - std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - -template -void TaskTable::initShards(RandomEngine &&random_engine) { - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto &shard_info : cluster_pull->getShardsInfo()) { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto &replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [](const TaskShardPtr &lhs, const TaskShardPtr &rhs) { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [](const TaskShardPtr &lhs, UInt8 is_remote) { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -} diff --git a/dbms/programs/copier/ZookeeperStaff.h b/dbms/programs/copier/ZookeeperStaff.h deleted file mode 100644 index 3133c68933d..00000000000 --- a/dbms/programs/copier/ZookeeperStaff.h +++ /dev/null @@ -1,224 +0,0 @@ -#pragma once - -/** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. - * We assume that we compare values that are not too far away. - * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. - */ -class WrappingUInt32 -{ -public: - UInt32 value; - - explicit WrappingUInt32(UInt32 _value) - : value(_value) - {} - - bool operator<(const WrappingUInt32 & other) const - { - return value != other.value && *this <= other; - } - - bool operator<=(const WrappingUInt32 & other) const - { - const UInt32 HALF = 1 << 31; - return (value <= other.value && other.value - value < HALF) - || (value > other.value && value - other.value > HALF); - } - - bool operator==(const WrappingUInt32 & other) const - { - return value == other.value; - } -}; - -/** Conforming Zxid definition. - * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions - * - * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html - * - * Actually here is the definition of Zxid. - * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). - * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid - * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. - */ -class Zxid -{ -public: - WrappingUInt32 epoch; - WrappingUInt32 counter; - explicit Zxid(UInt64 _zxid) - : epoch(_zxid >> 32) - , counter(_zxid) - {} - - bool operator<=(const Zxid & other) const - { - return (epoch < other.epoch) - || (epoch == other.epoch && counter <= other.counter); - } - - bool operator==(const Zxid & other) const - { - return epoch == other.epoch && counter == other.counter; - } -}; - -/* When multiple ClusterCopiers discover that the target partition is not empty, - * they will attempt to clean up this partition before proceeding to copying. - * - * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established - * based on a happens-before relation between the events. - * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. - * The fact of the partition hygiene is encoded by CleanStateClock. - * - * For you to know what mzxid means: - * - * ZooKeeper Stat Structure: - * The Stat structure for each znode in ZooKeeper is made up of the following fields: - * - * -- czxid - * The zxid of the change that caused this znode to be created. - * - * -- mzxid - * The zxid of the change that last modified this znode. - * - * -- ctime - * The time in milliseconds from epoch when this znode was created. - * - * -- mtime - * The time in milliseconds from epoch when this znode was last modified. - * - * -- version - * The number of changes to the data of this znode. - * - * -- cversion - * The number of changes to the children of this znode. - * - * -- aversion - * The number of changes to the ACL of this znode. - * - * -- ephemeralOwner - * The session id of the owner of this znode if the znode is an ephemeral node. - * If it is not an ephemeral node, it will be zero. - * - * -- dataLength - * The length of the data field of this znode. - * - * -- numChildren - * The number of children of this znode. - * */ - -class LogicalClock -{ -public: - std::optional zxid; - - LogicalClock() = default; - - explicit LogicalClock(UInt64 _zxid) - : zxid(_zxid) - {} - - bool hasHappened() const - { - return bool(zxid); - } - - /// happens-before relation with a reasonable time bound - bool happensBefore(const LogicalClock & other) const - { - return !zxid - || (other.zxid && *zxid <= *other.zxid); - } - - bool operator<=(const LogicalClock & other) const - { - return happensBefore(other); - } - - /// strict equality check - bool operator==(const LogicalClock & other) const - { - return zxid == other.zxid; - } -}; - - -class CleanStateClock -{ -public: - LogicalClock discovery_zxid; - std::optional discovery_version; - - LogicalClock clean_state_zxid; - std::optional clean_state_version; - - std::shared_ptr stale; - - bool is_clean() const - { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); - } - - bool is_stale() const - { - return stale->load(); - } - - CleanStateClock( - const zkutil::ZooKeeperPtr & zookeeper, - const String & discovery_path, - const String & clean_state_path) - : stale(std::make_shared(false)) - { - Coordination::Stat stat{}; - String _some_data; - auto watch_callback = - [stale = stale] (const Coordination::WatchResponse & rsp) - { - auto logger = &Poco::Logger::get("ClusterCopier"); - if (rsp.error == Coordination::ZOK) - { - switch (rsp.type) - { - case Coordination::CREATED: - LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); - stale->store(true); - break; - case Coordination::CHANGED: - LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); - stale->store(true); - } - } - }; - if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) - { - discovery_zxid = LogicalClock(stat.mzxid); - discovery_version = stat.version; - } - if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) - { - clean_state_zxid = LogicalClock(stat.mzxid); - clean_state_version = stat.version; - } - } - - bool operator==(const CleanStateClock & other) const - { - return !is_stale() - && !other.is_stale() - && discovery_zxid == other.discovery_zxid - && discovery_version == other.discovery_version - && clean_state_zxid == other.clean_state_zxid - && clean_state_version == other.clean_state_version; - } - - bool operator!=(const CleanStateClock & other) const - { - return !(*this == other); - } -}; From 6bc23f5eae2e8b2e12811802e91237333cd1dc66 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 12:01:06 +0300 Subject: [PATCH 012/147] merging --- dbms/programs/copier/Aliases.h | 3 + dbms/programs/copier/ClusterCopier.cpp | 8 +- dbms/programs/copier/ClusterCopier.h | 14 +-- dbms/programs/copier/Internals.cpp | 54 +++++++++ dbms/programs/copier/Internals.h | 25 ++++ dbms/programs/copier/ShardPartition.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 148 +++++++++++++++++------ 7 files changed, 206 insertions(+), 50 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index 4beff891bf4..d088d33f259 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -2,6 +2,7 @@ #include + namespace DB { @@ -10,12 +11,14 @@ namespace DB using DatabaseAndTableName = std::pair; /// Hierarchical description of the tasks + struct ShardPartitionPiece; struct ShardPartition; struct TaskShard; struct TaskTable; struct TaskCluster; struct ClusterPartition; + using PartitionPieces = std::vector; using TasksPartition = std::map>; using ShardInfo = Cluster::ShardInfo; using TaskShardPtr = std::shared_ptr; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 24ac6fdde0e..f133aadf6f2 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -578,7 +578,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } /// Removes MATERIALIZED and ALIAS columns from create table query -static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) +ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { const ASTs & column_asts = query_ast->as().columns_list->columns->children; auto new_columns = std::make_shared(); @@ -611,7 +611,7 @@ static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & qu } /// Replaces ENGINE and table name in a create query -static std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -919,7 +919,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Job for copying partition from particular shard. PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res; + PartitionTaskStatus res{Active}; try { @@ -941,7 +941,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo tryLogCurrentException(log, "An error occurred while updating the config"); } - return res; + return PartitionTaskStatus::Finished; } PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index e8ca88cb2a2..30a18b8b1a2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,20 +1,20 @@ #pragma once -#include -#include +#include "Aliases.h" #include "Internals.h" #include "TaskCluster.h" +#include "TaskTableAndShard.h" #include "ShardPartition.h" #include "ShardPartitionPiece.h" -#include "TaskTable.h" -#include "ZookeeperStaff.h" +#include "ZooKeeperStaff.h" + namespace DB { using ConfigurationPtr = Poco::AutoPtr; -static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) +ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { std::stringstream ss(xml_data); Poco::XML::InputSource input_source{ss}; @@ -92,10 +92,10 @@ public: size_t piece_number, const TasksShard & shards_with_partition); /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); + ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, + std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 05286515970..c7163d5eef0 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -109,6 +109,60 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } +ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + /// FIXME + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.primary_key) + return storage.primary_key->clone(); + + return storage.order_by->clone(); +} + +String createCommaSeparatedStringFrom(const Strings & strings) +{ + String answer; + for (auto & string: strings) + answer += string + ", "; + + /// Remove last comma and space + answer.pop_back(); + answer.pop_back(); + return answer; +} + +Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +{ + const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + + ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); + ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + + Strings answer; + answer.reserve(primary_key_or_order_by_arguments.size()); + + for (auto & column : primary_key_or_order_by_arguments) + answer.push_back(column->getColumnName()); + + return answer; +} + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { ShardPriority res; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 7177f6900de..5deca5d8d34 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -177,6 +177,31 @@ bool isExtendedDefinitionStorage(const ASTPtr & storage_ast); ASTPtr extractPartitionKey(const ASTPtr & storage_ast); +/* +* Choosing a Primary Key that Differs from the Sorting Key +* It is possible to specify a primary key (an expression with values that are written in the index file for each mark) +* that is different from the sorting key (an expression for sorting the rows in data parts). +* In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. +* This feature is helpful when using the SummingMergeTree and AggregatingMergeTree table engines. +* In a common case when using these engines, the table has two types of columns: dimensions and measures. +* Typical queries aggregate values of measure columns with arbitrary GROUP BY and filtering by dimensions. +* Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, +* it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns +* and this list must be frequently updated with newly added dimensions. +* In this case it makes sense to leave only a few columns in the primary key that will provide efficient +* range scans and add the remaining dimension columns to the sorting key tuple. +* ALTER of the sorting key is a lightweight operation because when a new column is simultaneously added t +* o the table and to the sorting key, existing data parts don't need to be changed. +* Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, +* the data is sorted by both the old and new sorting keys at the moment of table modification. +* +* */ +ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast); + +String createCommaSeparatedStringFrom(const Strings & strings); + +Strings extractPrimaryKeyString(const ASTPtr & storage_ast); + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); } diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index 03b3502efba..f7384b977e3 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -1,7 +1,7 @@ #pragma once -#include "Internals.h" -#include "TaskTable.h" +#include "Aliases.h" +#include "TaskTableAndShard.h" namespace DB { diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 05b1c3f543a..a73b7bfafa5 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -9,23 +9,58 @@ namespace DB struct TaskShard; -struct TaskTable -{ +struct TaskTable { TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); TaskCluster & task_cluster; + /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() + /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. + String getPartitionPath(const String & partition_name) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; + + String getCertainPartitionIsDirtyPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not Implemented"; + } + + String getCertainPartitionIsCleanedPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + String getCertainPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; String name_in_config; /// Used as task ID String table_id; + /// Column names in primary key + String primary_key_comma_separated; + /// Source cluster and table String cluster_pull_name; DatabaseAndTableName table_pull; @@ -35,14 +70,37 @@ struct TaskTable DatabaseAndTableName table_push; /// Storage of destination table + /// (tables that are stored on each shard of target cluster) String engine_push_str; ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; - /// A Distributed table definition used to split data + /* + * A Distributed table definition used to split data + * Distributed table will be created on each shard of default + * cluster to perform data copying and resharding + * */ String sharding_key_str; ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; + ASTPtr main_engine_split_ast; + + + /* + * Auxuliary table engines used to perform partition piece copying. + * Each AST represent table engine for certatin piece number. + * After copying partition piece is Ok, this piece will be moved to the main + * target table. All this tables are stored on each shard as the main table. + * We have to use separate tables for partition pieces because of the atomicity of copying. + * Also if we want to move some partition to another table, the partition keys have to be the same. + * */ + + + /* + * To copy partiton piece form one cluster to another we have to use Distributed table. + * In case of usage separate table (engine_push) for each partiton piece, + * we have to use many Distributed tables. + * */ + ASTs auxiliary_engine_split_asts; /// Additional WHERE expression to filter input data String where_condition_str; @@ -57,21 +115,26 @@ struct TaskTable Strings enabled_partitions; NameSet enabled_partitions_set; - /// Prioritized list of shards + /** + * Prioritized list of shards + * all_shards contains information about all shards in the table. + * So we have to check whether particular shard have current partiton or not while processing. + */ TasksShard all_shards; TasksShard local_shards; + /// All partitions of the current table. ClusterPartitions cluster_partitions; NameSet finished_cluster_partitions; /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String & partition_name) - { + ClusterPartition & getClusterPartition(const String &partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, + ErrorCodes::LOGICAL_ERROR); return it->second; } @@ -79,8 +142,8 @@ struct TaskTable UInt64 bytes_copied = 0; UInt64 rows_copied = 0; - template - void initShards(RandomEngine && random_engine); + template + void initShards(RandomEngine &&random_engine); }; @@ -121,36 +184,38 @@ struct TaskTable }; -inline String TaskTable::getPartitionPath(const String & partition_name) const +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 + assert(piece_number < number_of_splits); + return getPartitionPath(partition_name) + "/" + + std::to_string(piece_number); // 1...number_of_splits } -inline String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; + return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/shards"; } -inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) +inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, + const String & prefix_, const String & table_key) : task_cluster(parent) { String table_prefix = prefix_ + "." + table_key + "."; name_in_config = table_key; + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); @@ -170,18 +235,30 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); + + auxiliary_engine_split_asts.reserve(number_of_splits); { ParserExpressionWithOptionalAlias parser_expression(false); sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + sharding_key_ast); + + for (const auto piece_number : ext::range(0, number_of_splits)) + { + auxiliary_engine_split_asts.emplace_back + ( + createASTStorageDistributed(cluster_push_name, table_push.first, + table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + ); + } } where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { + if (!where_condition_str.empty()) { ParserExpressionWithOptionalAlias parser_expression(false); where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); @@ -192,31 +269,28 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf String enabled_partitions_prefix = table_prefix + "enabled_partitions"; has_enabled_partitions = config.has(enabled_partitions_prefix); - if (has_enabled_partitions) - { + if (has_enabled_partitions) { Strings keys; config.keys(enabled_partitions_prefix, keys); - if (keys.empty()) - { + if (keys.empty()) { /// Parse list of partition from space-separated string String partitions_str = config.getString(table_prefix + "enabled_partitions"); boost::trim_if(partitions_str, isWhitespaceASCII); boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { + } else { /// Parse sequence of ... - for (const String & key : keys) - { + for (const String &key : keys) { if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, + ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); } } - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + std::copy(enabled_partitions.begin(), enabled_partitions.end(), + std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); } } From b035dd815b24a6e27d05c63cfa5c0b959bde284f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 13:01:02 +0300 Subject: [PATCH 013/147] merge finished --- dbms/programs/copier/CMakeLists.txt | 2 +- dbms/programs/copier/ClusterCopier.cpp | 26 +++++----- dbms/programs/copier/ClusterCopier.h | 56 +++++++++------------- dbms/programs/copier/ClusterCopierApp.cpp | 1 - dbms/programs/copier/ClusterCopierApp.h | 3 ++ dbms/programs/copier/Internals.cpp | 3 +- dbms/programs/copier/Internals.h | 6 +++ dbms/programs/copier/ShardPartitionPiece.h | 16 +++---- dbms/programs/copier/TaskTableAndShard.h | 18 +++---- 9 files changed, 67 insertions(+), 64 deletions(-) diff --git a/dbms/programs/copier/CMakeLists.txt b/dbms/programs/copier/CMakeLists.txt index ff9ba2f250f..5573fbc5e5d 100644 --- a/dbms/programs/copier/CMakeLists.txt +++ b/dbms/programs/copier/CMakeLists.txt @@ -12,6 +12,6 @@ set(CLICKHOUSE_COPIER_LINK PRIVATE clickhouse_dictionaries string_utils ${Poco_XML_LIBRARY} PUBLIC daemon) -set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) +set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}) clickhouse_program_add(copier) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f133aadf6f2..65a82983d09 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -919,7 +919,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Job for copying partition from particular shard. PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res{Active}; + PartitionTaskStatus res; try { @@ -1324,7 +1324,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na interpreter.execute(); } -String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) +String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); Block block = getBlockWithAllStreamData(std::make_shared( @@ -1347,7 +1347,8 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time } /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. -void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) +void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, bool create_split, const size_t piece_number) { TaskTable & task_table = task_shard.task_table; @@ -1423,14 +1424,16 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti return res; } -bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) +bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name) { createShardInternalTables(timeouts, task_shard, false); TaskTable & task_table = task_shard.task_table; std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + + " = (" + partition_quoted_name + " AS partition_key))"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1451,14 +1454,15 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); /// Just copypaste the function above bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) { createShardInternalTables(timeouts, task_shard, false); TaskTable & task_table = task_shard.task_table; std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + + " = (" + partition_quoted_name + " AS partition_key))"; const size_t number_of_splits = task_table.number_of_splits; const String & primary_key_comma_separated = task_table.primary_key_comma_separated; @@ -1495,10 +1499,10 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, - PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const + const ASTPtr & query_ast_, + const Settings * settings, + PoolMode pool_mode, + UInt64 max_successful_executions_per_shard) const { auto num_shards = cluster->getShardsInfo().size(); std::vector per_shard_num_successful_replicas(num_shards, 0); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 30a18b8b1a2..0555b2d5c04 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -12,15 +12,6 @@ namespace DB { -using ConfigurationPtr = Poco::AutoPtr; - -ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) -{ - std::stringstream ss(xml_data); - Poco::XML::InputSource input_source{ss}; - return {new Poco::Util::XMLConfiguration{&input_source}}; -} - class ClusterCopier { public: @@ -34,9 +25,7 @@ public: host_id(std::move(host_id_)), working_database_name(std::move(proxy_database_name_)), context(context_), - log(&Poco::Logger::get("ClusterCopier")) - { - } + log(&Poco::Logger::get("ClusterCopier")) {} void init(); @@ -57,17 +46,32 @@ public: void process(const ConnectionTimeouts & timeouts); /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true); + void setSafeMode(bool is_safe_mode_ = true) + { + is_safe_mode = is_safe_mode_; + } - void setCopyFaultProbability(double copy_fault_probability_); + void setCopyFaultProbability(double copy_fault_probability_) + { + copy_fault_probability = copy_fault_probability_; + } - protected: +protected: - String getWorkersPath() const; + String getWorkersPath() const + { + return task_cluster->task_zookeeper_path + "/task_active_workers"; + } - String getWorkersPathVersion() const; + String getWorkersPathVersion() const + { + return getWorkersPath() + "_version"; + } - String getCurrentWorkerNodePath() const; + String getCurrentWorkerNodePath() const + { + return getWorkersPath() + "/" + host_id; + } zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, @@ -105,22 +109,6 @@ public: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); - /// Execution status of a task - enum class PartitionTaskStatus - { - Active, - Finished, - Error, - }; - - - enum class PartititonPieceTaskStatus - { - Active, - Finished, - Error, - }; - /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index 37b02296086..e6ff4c521ef 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -1,5 +1,4 @@ #include "ClusterCopierApp.h" -#include "ClusterCopier.h" namespace DB { diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h index fe228fd6194..25350aefd3a 100644 --- a/dbms/programs/copier/ClusterCopierApp.h +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -1,7 +1,10 @@ #pragma once + #include #include +#include "ClusterCopier.h" + /* clickhouse cluster copier util * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. * diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index c7163d5eef0..173a5dc3c59 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -2,6 +2,8 @@ namespace DB { +using ConfigurationPtr = Poco::AutoPtr; + ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { std::stringstream ss(xml_data); @@ -9,7 +11,6 @@ ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) return {new Poco::Util::XMLConfiguration{&input_source}}; } - String getQuotedTable(const String & database, const String & table) { if (database.empty()) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 5deca5d8d34..2937171903c 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -155,6 +155,12 @@ enum class PartitionTaskStatus Error, }; +enum class PartititonPieceTaskStatus +{ + Active, + Finished, + Error, +}; struct MultiTransactionInfo { diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 303407d1d5b..f7ae8013b47 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -34,43 +34,43 @@ struct ShardPartitionPiece { }; -String ShardPartitionPiece::getPartitionPiecePath() const +inline String ShardPartitionPiece::getPartitionPiecePath() const { return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); } -String ShardPartitionPiece::getPartitionPieceCleanStartPath() const +inline String ShardPartitionPiece::getPartitionPieceCleanStartPath() const { return getPartitionPiecePath() + "/clean_start"; } -String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const +inline String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const { return getPartitionPiecePath() + "/is_dirty"; } -String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const +inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const { return getPartitionPieceIsDirtyPath() + "/is_cleaned"; } -String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const +inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const { return getPartitionPiecePath() + "/partition_active_workers"; } -String ShardPartitionPiece::getActiveWorkerPath() const +inline String ShardPartitionPiece::getActiveWorkerPath() const { return getPartitionPieceActiveWorkersPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } /// On what shards do we have current partition. -String ShardPartitionPiece::getPartitionPieceShardsPath() const +inline String ShardPartitionPiece::getPartitionPieceShardsPath() const { return getPartitionPiecePath() + "/shards"; } -String ShardPartitionPiece::getShardStatusPath() const +inline String ShardPartitionPiece::getShardStatusPath() const { return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index a73b7bfafa5..806b6b66cd4 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -184,25 +184,27 @@ struct TaskTable { }; -inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const -{ +inline String TaskTable::getPartitionPath(const String &partition_name) const { + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { assert(piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + std::to_string(piece_number); // 1...number_of_splits } -inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/shards"; } From 1be4a35f15c74ca327668185fd3f24bd9865c2b7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 20:26:20 +0300 Subject: [PATCH 014/147] successful copying --- dbms/programs/copier/ClusterCopier.cpp | 112 +++++++++++++-------- dbms/programs/copier/ClusterCopier.h | 4 +- dbms/programs/copier/ShardPartitionPiece.h | 2 +- dbms/programs/copier/TaskTableAndShard.h | 13 ++- dbms/programs/copier/ZooKeeperStaff.h | 7 +- 5 files changed, 81 insertions(+), 57 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 65a82983d09..d6859b54cbf 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -17,7 +17,7 @@ void ClusterCopier::init() { if (response.error != Coordination::ZOK) return; - UInt64 version = ++task_descprtion_version; + UInt64 version = ++task_description_version; LOG_DEBUG(log, "Task description should be updated, local version " << version); }; @@ -227,9 +227,9 @@ void ClusterCopier::reloadTaskDescription() void ClusterCopier::updateConfigIfNeeded() { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + UInt64 version_to_update = task_description_version; + bool is_outdated_version = task_description_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); if (!is_outdated_version && !is_expired_session) return; @@ -237,7 +237,7 @@ void ClusterCopier::updateConfigIfNeeded() LOG_DEBUG(log, "Updating task description"); reloadTaskDescription(); - task_descprtion_current_version = version_to_update; + task_description_current_version = version_to_update; } void ClusterCopier::process(const ConnectionTimeouts & timeouts) @@ -312,6 +312,13 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) /// Protected section + +/* + * Creates task worker node and checks maximum number of workers not to exceed the limit. + * To achive this we have to check version of workers_version_path node and create current_worker_path + * node atomically. + * */ + zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, const String & description, @@ -324,8 +331,8 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee std::this_thread::sleep_for(current_sleep_time); String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); UInt64 num_bad_version_errors = 0; @@ -629,6 +636,9 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A return res; } + +/// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. + bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) { if (is_safe_mode) @@ -636,11 +646,11 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut TaskTable & task_table = task_partition.task_shard.task_table; - const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_shards_path = task_partition.getPartitionShardsPath(); const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try @@ -957,12 +967,15 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn } -PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task) + +/*...*/ +PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( + const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) { TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; const size_t number_of_splits = task_table.number_of_splits; @@ -974,15 +987,15 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio auto zookeeper = context.getZooKeeper(); - const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); - const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); - const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); /// Auxiliary functions: /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + auto create_is_dirty_node = [&] (const CleanStateClock & clock) { if (clock.is_stale()) LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); @@ -1001,16 +1014,17 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio }; /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false) + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, bool enable_splitting, String limit = "") { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + + " = " + toString(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1031,23 +1045,26 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio LogicalClock task_start_clock; { Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + if (zookeeper->exists(partition_piece.getPartitionPieceShardsPath(), &stat)) task_start_clock = LogicalClock(stat.mzxid); } - /// Do not start if partition is dirty, try to clean it + /// Do not start if partition piece is dirty, try to clean it if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + LOG_DEBUG(log, "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " appears to be clean"); zookeeper->createAncestors(current_task_piece_status_path); } else { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + LOG_DEBUG(log, "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " is dirty, try to drop it"); try { + /// TODO: tryDropPartitionPiece. tryDropPartition(task_partition, zookeeper, clean_state_clock); } catch (...) @@ -1085,13 +1102,16 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); + LOG_DEBUG(log, "Task " << current_task_piece_status_path + << " has been successfully executed by " << status.owner); return PartitionTaskStatus::Finished; } - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); + /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. + /// Initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_piece_status_path + << " has not been successfully finished by " << status.owner + << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; @@ -1101,13 +1121,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio /// Check that destination partition is empty if we are first worker /// NOTE: this check is incorrect if pull and push tables have different partition key! String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + if (!zookeeper->tryGet(partition_piece.getPartitionPieceCleanStartPath(), clean_start_status) || clean_start_status != "ok") { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + zookeeper->createIfNotExists(partition_piece.getPartitionPieceCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + /// TODO: Why table_split_shard??? + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()", /*enable_splitting*/ true); UInt64 count; { Context local_context = context; @@ -1122,20 +1143,21 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio if (count != 0) { Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); + zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning if (stat_shards.numChildren == 0) { LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); + << " piece " << toString(current_piece_number) + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + zookeeper->set(partition_piece.getPartitionPieceCleanStartPath(), "ok"); } /// At this point, we need to sync that the destination table is clean /// before any actual work @@ -1146,12 +1168,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " clean state changed, cowardly bailing"); return PartitionTaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " is dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } @@ -1163,7 +1187,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio /// Define push table for current partition piece auto database_and_table_for_current_piece= std::pair( task_table.table_push.first, - task_table.table_push.second + ".piece_" + toString(current_piece_number)); + task_table.table_push.second + "_piece_" + toString(current_piece_number)); auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, database_and_table_for_current_piece, task_table.engine_push_ast); @@ -1174,8 +1198,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, create_query_push_ast, &task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) + << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } /// Do the copying @@ -1188,7 +1212,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio } // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ inject_fault ? "1" : ""); LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() << " : " << queryToString(query_select_ast)); @@ -1361,7 +1385,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); + working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 0555b2d5c04..41fdc326ed2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -156,8 +156,8 @@ private: String working_database_name; /// Auto update config stuff - UInt64 task_descprtion_current_version = 1; - std::atomic task_descprtion_version{1}; + UInt64 task_description_current_version = 1; + std::atomic task_description_version{1}; Coordination::WatchCallback task_description_watch_callback; /// ZooKeeper session used to set the callback zkutil::ZooKeeperPtr task_description_watch_zookeeper; diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index f7ae8013b47..a99221b8a97 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -56,7 +56,7 @@ inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const { - return getPartitionPiecePath() + "/partition_active_workers"; + return getPartitionPiecePath() + "/partition_piece_active_workers"; } inline String ShardPartitionPiece::getActiveWorkerPath() const diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index d585fb184ed..c0795340e47 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -20,11 +20,11 @@ struct TaskTable { String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; + String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsDirtyPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -33,7 +33,7 @@ struct TaskTable { String getCertainPartitionIsCleanedPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -42,7 +42,7 @@ struct TaskTable { String getCertainPartitionTaskStatusPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -181,7 +181,10 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; + DatabaseAndTableName table_split_shard; + + std::vector list_of_split_tables_on_shard; }; @@ -255,7 +258,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf auxiliary_engine_split_asts.emplace_back ( createASTStorageDistributed(cluster_push_name, table_push.first, - table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + table_push.second + "_piece_" + toString(piece_number), sharding_key_ast) ); } } diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index 3133c68933d..2fc4d35400d 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,11 +157,8 @@ public: bool is_clean() const { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); + return !is_stale() + && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } bool is_stale() const From 2869c015712f444dcce8a2c7f9c321da8ae112d8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 21:58:00 +0300 Subject: [PATCH 015/147] drop partition piece --- dbms/programs/copier/ClusterCopier.cpp | 32 ++++++++++++++-------- dbms/programs/copier/ClusterCopier.h | 3 +- dbms/programs/copier/ShardPartitionPiece.h | 7 +++++ 3 files changed, 29 insertions(+), 13 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d6859b54cbf..cde272074da 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -639,18 +639,23 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A /// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. -bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) +bool ClusterCopier::tryDropPartitionPiece( + ShardPartition & task_partition, + const size_t current_piece_number, + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock) { if (is_safe_mode) throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); TaskTable & task_table = task_partition.task_shard.task_table; + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String current_shards_path = partition_piece.getPartitionPieceShardsPath(); + const String current_partition_active_workers_dir = partition_piece.getPartitionPieceActiveWorkersPath(); + const String is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String dirty_cleaner_path = partition_piece.getPartitionPieceCleanerPath(); + const String is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try @@ -661,7 +666,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut { if (e.code == Coordination::ZNODEEXISTS) { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " is cleaning now by somebody, sleep"); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -674,7 +680,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut { if (stat.numChildren != 0) { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren + << " active workers while trying to drop it. Going to sleep."); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -703,7 +710,7 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut // Lock the dirty flag zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + zookeeper->tryRemove(partition_piece.getPartitionPieceCleanStartPath()); CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); /// Remove all status nodes @@ -716,7 +723,7 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut } } - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push) + "_piece_" + toString(current_piece_number); query += " DROP PARTITION " + task_partition.name + ""; /// TODO: use this statement after servers will be updated up to 1.1.54310 @@ -754,7 +761,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut return false; } - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) + << " was dropped on cluster " << task_table.cluster_push_name); if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) zookeeper->set(current_shards_path, host_id); } @@ -1065,7 +1073,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( try { /// TODO: tryDropPartitionPiece. - tryDropPartition(task_partition, zookeeper, clean_state_clock); + tryDropPartitionPiece(task_partition, current_piece_number, zookeeper, clean_state_clock); } catch (...) { diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 41fdc326ed2..2729b295bce 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -102,7 +102,8 @@ protected: std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); + bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number, + const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index a99221b8a97..aeaa24fef13 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -27,6 +27,8 @@ struct ShardPartitionPiece { [[maybe_unused]] String getShardStatusPath() const; + String getPartitionPieceCleanerPath() const; + bool is_absent_piece; const size_t current_piece_number; @@ -75,5 +77,10 @@ inline String ShardPartitionPiece::getShardStatusPath() const return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } +inline String ShardPartitionPiece::getPartitionPieceCleanerPath() const +{ + return getPartitionPieceIsDirtyPath() + "/cleaner"; +} + } From de49d00079e31f6250adad9effa3e648137bbb52 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 21 Feb 2020 18:21:31 +0300 Subject: [PATCH 016/147] Use re2 instead of regexp. Use PeekableReadBuffer insted of common ReadBuffer. --- .../Formats/Impl/RegexpRowInputFormat.cpp | 59 ++++++++++++++----- .../Formats/Impl/RegexpRowInputFormat.h | 15 ++++- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index f0f3d10755b..387f81f1155 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -1,10 +1,8 @@ -#include #include #include #include #include - -#include +#include namespace DB { @@ -17,8 +15,20 @@ namespace ErrorCodes RegexpRowInputFormat::RegexpRowInputFormat( ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) + : IRowInputFormat(header_, in_, std::move(params_)), buf(in_), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) { + size_t fields_count = regexp.NumberOfCapturingGroups(); + matched_fields.resize(fields_count); + re2_arguments.resize(fields_count); + re2_arguments_ptrs.resize(fields_count); + for (size_t i = 0; i != fields_count; ++i) + { + // Bind an argument to a matched field. + re2_arguments[i] = &matched_fields[i]; + // Save pointer to argument. + re2_arguments_ptrs[i] = &re2_arguments[i]; + } + field_format = stringToFormat(format_settings_.regexp.escaping_rule); } @@ -40,7 +50,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) const auto & type = getPort().getHeader().getByPosition(index).type; bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); bool read = true; - ReadBuffer field_buf(matched_fields[index + 1].first, matched_fields[index + 1].length(), 0); + ReadBuffer field_buf(matched_fields[index].data(), matched_fields[index].size(), 0); try { switch (field_format) @@ -82,7 +92,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) { - if (matched_fields.size() != columns.size() + 1) + if (matched_fields.size() != columns.size()) throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA); ext.read_columns.assign(columns.size(), false); @@ -94,23 +104,44 @@ void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowRead bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) { - if (in.eof()) + if (buf.eof()) return false; - char * line_end = find_first_symbols<'\n', '\r'>(in.position(), in.buffer().end()); - bool match = std::regex_match(in.position(), line_end, matched_fields, regexp); + PeekableReadBufferCheckpoint checkpoint{buf}; + + size_t line_size = 0; + + while (!buf.eof() && *buf.position() != '\n' && *buf.position() != '\r') + { + ++buf.position(); + ++line_size; + } + + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + + bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size()); + bool read_line = true; if (!match) { if (!format_settings.regexp.skip_unmatched) - throw Exception("Line \"" + std::string(in.position(), line_end) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); - in.position() = line_end + 1; - return true; + throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); + read_line = false; } - readFieldsFromMatch(columns, ext); + if (read_line) + readFieldsFromMatch(columns, ext); + + buf.position() += line_size; + + // Two sequential increments are needed to support DOS-style newline ("\r\n"). + if (!buf.eof() && *buf.position() == '\r') + ++buf.position(); + + if (!buf.eof() && *buf.position() == '\n') + ++buf.position(); - in.position() = line_end + 1; return true; } diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 48a711d0cde..218bb71055f 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -1,10 +1,13 @@ #pragma once -#include +#include +#include +#include #include #include #include #include +#include namespace DB { @@ -34,10 +37,16 @@ private: void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); FieldFormat stringToFormat(const String & format); + PeekableReadBuffer buf; const FormatSettings format_settings; - std::regex regexp; - std::match_results matched_fields; FieldFormat field_format; + + RE2 regexp; + // The vector of fields extracted from line using regexp. + std::vector matched_fields; + // These two vectors are needed to use RE2::FullMatchN (function for extracting fields). + std::vector re2_arguments; + std::vector re2_arguments_ptrs; }; } From 0b4d8bdb08bb7de45f893dc5a6d8212b51afc67b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:00:50 +0300 Subject: [PATCH 017/147] move partition added --- dbms/programs/copier/Aliases.h | 1 + dbms/programs/copier/ClusterCopier.cpp | 287 ++++++++++++--------- dbms/programs/copier/ClusterCopier.h | 18 +- dbms/programs/copier/ShardPartitionPiece.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 60 +++-- 5 files changed, 212 insertions(+), 158 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index d088d33f259..fff57dd603f 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -9,6 +9,7 @@ namespace DB using ConfigurationPtr = Poco::AutoPtr; using DatabaseAndTableName = std::pair; + using ListOfDatabasesAndTableNames = std::vector; /// Hierarchical description of the tasks struct ShardPartitionPiece; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index cde272074da..e9dc613f85e 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -386,99 +386,32 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee } } -/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. - * State of some task could change during the processing. - * We have to ensure that all shards have the finished state and there is no dirty flag. - * Moreover, we have to check status twice and check zxid, because state can change during the checking. - */ -bool ClusterCopier::checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + +bool ClusterCopier::checkPartitionPieceIsClean( + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock, + const String & task_status_path) const { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + LogicalClock task_start_clock; - auto zookeeper = context.getZooKeeper(); + Coordination::Stat stat{}; + if (zookeeper->exists(task_status_path, &stat)) + task_start_clock = LogicalClock(stat.mzxid); - Strings status_paths; - for (auto & shard : shards_with_partition) + /// If statement for readability. + if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + return true; } - - std::vector zxid1, zxid2; - - try + else { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); return false; } - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; } -bool ClusterCopier::checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + +bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) { bool answer = true; for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) @@ -496,7 +429,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons size_t piece_number, const TasksShard & shards_with_partition) { LOG_DEBUG(log, "Check that all shards processed partition " << partition_name - << " piece number" + toString(piece_number) + " successfully"); + << " piece " + toString(piece_number) + " successfully"); auto zookeeper = context.getZooKeeper(); @@ -525,30 +458,42 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); if (status.state != TaskState::Finished) { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + LOG_INFO(log, "The task " << res.data << " is being rewritten by " + << status.owner << ". Partition piece will be rechecked"); return false; } zxid1.push_back(res.stat.pzxid); } - // Check that partition is not dirty + const String piece_is_dirty_flag_path = task_table.getCertainPartitionPieceIsDirtyPath(partition_name, piece_number); + const String piece_is_dirty_cleaned_path = task_table.getCertainPartitionPieceIsCleanedPath(partition_name, piece_number); + const String piece_task_status_path = task_table.getCertainPartitionPieceTaskStatusPath(partition_name, piece_number); + + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + std::cout << piece_is_dirty_flag_path << std::endl; + std::cout << piece_is_dirty_cleaned_path << std::endl; + std::cout << piece_task_status_path << std::endl; + + const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); + + for (size_t i = 0; i < 10; ++i) { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path)) { - LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); - return false; + std::cout << "clean" << std::endl; } + else + { + std::cout << "dirty" << std::endl; + } + } + + if (!is_clean) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; } get_futures.clear(); @@ -565,7 +510,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons catch (const Coordination::Exception & e) { LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " - << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); return false; } @@ -581,7 +526,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); return true; - } /// Removes MATERIALIZED and ALIAS columns from create table query @@ -637,8 +581,6 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A } -/// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. - bool ClusterCopier::tryDropPartitionPiece( ShardPartition & task_partition, const size_t current_piece_number, @@ -886,7 +828,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab { partition_is_done = !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + || (partition_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); } catch (...) { @@ -988,10 +930,11 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( const size_t number_of_splits = task_table.number_of_splits; const String primary_key_comma_separated = task_table.primary_key_comma_separated; - UNUSED(number_of_splits); - UNUSED(partition_piece); + /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard, true, current_piece_number); + createShardInternalTables(timeouts, task_shard, true); + + auto split_table_for_current_piece = task_shard.list_of_split_tables_on_shard[current_piece_number]; auto zookeeper = context.getZooKeeper(); @@ -1029,7 +972,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - if (enable_splitting) query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + " = " + toString(current_piece_number) + " )"; @@ -1048,18 +990,30 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + const String piece_status_path = partition_piece.getPartitionPieceShardsPath(); - LogicalClock task_start_clock; + CleanStateClock clean_state_clock(zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + std::cout << piece_is_dirty_flag_path << std::endl; + std::cout << piece_is_dirty_cleaned_path << std::endl; + std::cout << piece_status_path << std::endl; + + const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); + + for (size_t i = 0; i < 10; ++i) { - Coordination::Stat stat{}; - if (zookeeper->exists(partition_piece.getPartitionPieceShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); + if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path)) + { + std::cout << "clean" << std::endl; + } + else + { + std::cout << "dirty" << std::endl; + } } /// Do not start if partition piece is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + if (is_clean) { LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + toString(current_piece_number) + " appears to be clean"); @@ -1136,7 +1090,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( *zookeeper, host_id); // Maybe we are the first worker /// TODO: Why table_split_shard??? - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()", /*enable_splitting*/ true); + + ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; { Context local_context = context; @@ -1228,7 +1183,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( ASTPtr query_insert_ast; { String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES "; ParserQuery p_query(query.data() + query.size()); query_insert_ast = parseQuery(p_query, query, 0); @@ -1314,25 +1269,83 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } } + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " copied. But not moved to original destination table."); + + + /// Try create original table (if not exists) on each shard + { + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) + << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Move partition to original destination table. + { + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << task_partition.name + << " piece " << toString(current_piece_number) << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + query_alter_ast_string += "ALTER TABLE " + getQuotedTable(helping_table) + + " MOVE PARTITION " + task_partition.name + + " TO TABLE " + getQuotedTable(original_table); + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try + { + UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_ONE, 1); + + LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << task_partition.name + << " piece " << toString(current_piece_number) << "to original table"); + throw; + } + + } + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " clean state changed, cowardly bailing"); return PartitionTaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " became dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } zookeeper->set(current_task_piece_status_path, state_finished, 0); } - LOG_INFO(log, "Partition " << task_partition.name << " copied"); + /// TODO: LOG_INFO (Piece copied and moved to destination table) + + + return PartitionTaskStatus::Finished; } @@ -1380,7 +1393,7 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, bool create_split, const size_t piece_number) + TaskShard & task_shard, bool create_split) { TaskTable & task_table = task_shard.task_table; @@ -1392,8 +1405,13 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); + task_shard.main_table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + + for (const auto & piece_number : ext::range(0, task_table.number_of_splits)) + { + task_shard.list_of_split_tables_on_shard[piece_number] = + DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); + } /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; @@ -1401,16 +1419,35 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); dropAndCreateLocalTable(create_table_pull_ast); if (create_split) + { + auto create_table_split_piece_ast = rewriteCreateQueryStorage( + create_query_ast, + task_shard.main_table_split_shard, + task_table.main_engine_split_ast); + dropAndCreateLocalTable(create_table_split_piece_ast); + + /// Create auxilary split tables for each piece + for (const auto & piece_number : ext::range(0, task_table.number_of_splits)) + { + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; + + create_table_split_piece_ast = rewriteCreateQueryStorage( + create_query_ast, + task_shard.list_of_split_tables_on_shard[piece_number], + storage_piece_split_ast); + + dropAndCreateLocalTable(create_table_split_piece_ast); + } + } + } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 2729b295bce..a5587045299 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -78,14 +78,23 @@ protected: const String & description, bool unprioritized); + /* + * Checks that partition piece or some other entity is clean. + * The only requirement is that you have to pass is_dirty_flag_path and is_dirty_cleaned_path to the function. + * And is_dirty_flag_path is a parent of is_dirty_cleaned_path. + * */ + bool checkPartitionPieceIsClean( + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock, + const String & task_status_path) const; + + bool checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); + /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. * State of some task could change during the processing. * We have to ensure that all shards have the finished state and there is no dirty flag. * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); - - bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); /* The same as function above * Assume that we don't know on which shards do we have partition certain piece. @@ -128,7 +137,8 @@ protected: ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0); + /// TODO: rewrite comment + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index aeaa24fef13..04f7f458a00 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -38,7 +38,7 @@ struct ShardPartitionPiece { inline String ShardPartitionPiece::getPartitionPiecePath() const { - return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); + return shard_partition.getPartitionPath() + "/piece_" + toString(current_piece_number); } inline String ShardPartitionPiece::getPartitionPieceCleanStartPath() const @@ -53,7 +53,7 @@ inline String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const { - return getPartitionPieceIsDirtyPath() + "/is_cleaned"; + return getPartitionPieceIsDirtyPath() + "/cleaned"; } inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index c0795340e47..a1b7620bbcd 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -24,30 +24,15 @@ struct TaskTable { String getCertainPartitionIsDirtyPath(const String & partition_name) const; - String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not Implemented"; - } + String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsCleanedPath(const String & partition_name) const; - String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } + String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionTaskStatusPath(const String & partition_name) const; - String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } + String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -149,7 +134,10 @@ struct TaskTable { struct TaskShard { - TaskShard(TaskTable &parent, const ShardInfo &info_) : task_table(parent), info(info_) {} + TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) + { + list_of_split_tables_on_shard.assign(task_table.number_of_splits, DatabaseAndTableName()); + } TaskTable & task_table; @@ -182,9 +170,9 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; - DatabaseAndTableName table_split_shard; + DatabaseAndTableName main_table_split_shard; - std::vector list_of_split_tables_on_shard; + ListOfDatabasesAndTableNames list_of_split_tables_on_shard; }; @@ -194,24 +182,42 @@ inline String TaskTable::getPartitionPath(const String &partition_name) const { + "/" + escapeForFileName(partition_name); // 201701 } -inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const +{ assert(piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + - std::to_string(piece_number); // 1...number_of_splits + return getPartitionPath(partition_name) + "/piece_" + toString(piece_number); // 1...number_of_splits } -inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const +{ return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const +{ + return getPartitionPiecePath(partition_name, piece_number) + "/is_dirty"; +} + +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const +{ return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const +{ + return getCertainPartitionPieceIsDirtyPath(partition_name, piece_number) + "/cleaned"; +} + +inline String TaskTable::getCertainPartitionTaskStatusPath(const String & partition_name) const +{ return getPartitionPath(partition_name) + "/shards"; } +inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const +{ + return getPartitionPiecePath(partition_name, piece_number) + "/shards"; +} + inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, const String & table_key) : task_cluster(parent) From 16322b13528eb4ff52412784c453fa8527068513 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:18:16 +0300 Subject: [PATCH 018/147] replace partition --- dbms/programs/copier/ClusterCopier.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index e9dc613f85e..bd4c15b6fdb 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1301,9 +1301,9 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += "ALTER TABLE " + getQuotedTable(helping_table) + - " MOVE PARTITION " + task_partition.name + - " TO TABLE " + getQuotedTable(original_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " REPLACE PARTITION " + task_partition.name + + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); From 5e01f348c57adc2ddebb860b5a126019e550519b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:47:43 +0300 Subject: [PATCH 019/147] return back alter move --- dbms/programs/copier/ClusterCopier.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index bd4c15b6fdb..1ab137ca141 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1301,9 +1301,9 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " REPLACE PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(helping_table) + + " MOVE PARTITION " + task_partition.name + + " TO TABLE " + getQuotedTable(original_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); From 187872517f2f924b4ebd4bd3dc264459b9200e57 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 21 Feb 2020 21:49:18 +0300 Subject: [PATCH 020/147] Add DOS-style new line in FileSegmentationEngine. --- .../Formats/Impl/RegexpRowInputFormat.cpp | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 387f81f1155..90d1cf99b2e 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -85,6 +86,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) } catch (Exception & e) { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); throw; } return read; @@ -160,17 +162,21 @@ void registerInputFormatProcessorRegexp(FormatFactory & factory) static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { char * pos = in.position(); - bool need_more_data = true; - while (loadAtPosition(in, memory, pos) && need_more_data) + while (loadAtPosition(in, memory, pos) && (memory.size() + static_cast(pos - in.position()) >= min_chunk_size)) { pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end()); if (pos == in.buffer().end()) continue; - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) - need_more_data = false; - ++pos; + // Support DOS-style newline ("\r\n") + if (*pos++ == '\r') + { + if (pos == in.buffer().end()) + loadAtPosition(in, memory, pos); + if (*pos == '\n') + ++pos; + } } saveUpToPosition(in, memory, pos); From 1b8174b9d068317cab8b1b1db88cd652c90d6922 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 23 Feb 2020 17:25:55 +0300 Subject: [PATCH 021/147] trigger ci --- dbms/programs/copier/Aliases.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index fff57dd603f..d91685445f8 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -4,7 +4,6 @@ namespace DB - { using ConfigurationPtr = Poco::AutoPtr; From 476d25b6b27002678becf3e989e7cfa5c8237988 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 25 Feb 2020 15:38:11 +0300 Subject: [PATCH 022/147] fix build --- dbms/programs/copier/ClusterCopier.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 1ab137ca141..c6409552136 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -901,7 +901,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo tryLogCurrentException(log, "An error occurred while updating the config"); } - return PartitionTaskStatus::Finished; + return res; } PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, @@ -909,12 +909,17 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - /// ThreadPool maybe ?? + PartitionTaskStatus res; + PartitionTaskStatus answer = PartitionTaskStatus::Finished; + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) - processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - - return PartitionTaskStatus::Finished; + { + res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + if (res == PartitionTaskStatus::Error) + answer = res; + } + return answer; } From 9591678f84172bcb9246a876c6ac1031c5375f0f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 26 Feb 2020 14:44:21 +0300 Subject: [PATCH 023/147] useledd --- dbms/programs/copier/Internals.cpp | 10 ++++++++-- dbms/programs/copier/TaskTableAndShard.h | 2 ++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 93be1ea00e8..b0d54a65fee 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -73,6 +73,8 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); + std::cout << "inside extractPartitionKey " << storage_str << std::endl; + const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); @@ -115,7 +117,12 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - const auto & storage = storage_ast->as(); + ParserStorage parser_storage; + auto new_storage_ast = parseQuery(parser_storage, storage_str, 0); + + std::cout << "inside extractPrimaryKeyOrOrderBy" << storage_str << std::endl; + + const auto & storage = new_storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) @@ -124,7 +131,6 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) ErrorCodes::BAD_ARGUMENTS); } - /// FIXME if (!isExtendedDefinitionStorage(storage_ast)) { throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index a1b7620bbcd..a8590d54b9b 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -246,7 +246,9 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf { ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + std::cout << engine_push_str << std::endl; primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } From fec76334cc07c19acd427021edcefd2ed27008bf Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 27 Feb 2020 15:33:56 +0300 Subject: [PATCH 024/147] Rename tests. --- ...input_format.reference => 01085_regexp_input_format.reference} | 0 ...{01079_regexp_input_format.sh => 01085_regexp_input_format.sh} | 0 ...ference => 01086_regexp_input_format_skip_unmatched.reference} | 0 ...p_unmatched.sh => 01086_regexp_input_format_skip_unmatched.sh} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{01079_regexp_input_format.reference => 01085_regexp_input_format.reference} (100%) rename dbms/tests/queries/0_stateless/{01079_regexp_input_format.sh => 01085_regexp_input_format.sh} (100%) rename dbms/tests/queries/0_stateless/{01080_regexp_input_format_skip_unmatched.reference => 01086_regexp_input_format_skip_unmatched.reference} (100%) rename dbms/tests/queries/0_stateless/{01080_regexp_input_format_skip_unmatched.sh => 01086_regexp_input_format_skip_unmatched.sh} (100%) diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference b/dbms/tests/queries/0_stateless/01085_regexp_input_format.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01079_regexp_input_format.reference rename to dbms/tests/queries/0_stateless/01085_regexp_input_format.reference diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01085_regexp_input_format.sh similarity index 100% rename from dbms/tests/queries/0_stateless/01079_regexp_input_format.sh rename to dbms/tests/queries/0_stateless/01085_regexp_input_format.sh diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference b/dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference rename to dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.reference diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh similarity index 100% rename from dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh rename to dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh From aa0d44131d958c6545cee7ae3cbef6d680ef7969 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 12:49:06 +0300 Subject: [PATCH 025/147] normal extracting --- dbms/programs/copier/ClusterCopier.cpp | 23 ------ dbms/programs/copier/Internals.cpp | 89 +++++++++++++++++++----- dbms/programs/copier/Internals.h | 8 ++- dbms/programs/copier/TaskTableAndShard.h | 2 +- 4 files changed, 77 insertions(+), 45 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 90ea75708ce..e3013c5bebe 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -486,17 +486,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); - for (size_t i = 0; i < 10; ++i) - { - if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path)) - { - std::cout << "clean" << std::endl; - } - else - { - std::cout << "dirty" << std::endl; - } - } if (!is_clean) { @@ -1013,18 +1002,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); - for (size_t i = 0; i < 10; ++i) - { - if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path)) - { - std::cout << "clean" << std::endl; - } - else - { - std::cout << "dirty" << std::endl; - } - } - /// Do not start if partition piece is dirty, try to clean it if (is_clean) { diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index e16b55bebb0..a0beccbde2c 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -1,4 +1,5 @@ #include "Internals.h" +#include namespace DB { @@ -117,16 +118,11 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } -ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +ASTPtr extractPrimaryKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - ParserStorage parser_storage; - auto new_storage_ast = parseQuery(parser_storage, storage_str, 0); - - std::cout << "inside extractPrimaryKeyOrOrderBy" << storage_str << std::endl; - - const auto & storage = new_storage_ast->as(); + const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) @@ -144,10 +140,37 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) if (storage.primary_key) return storage.primary_key->clone(); - return storage.order_by->clone(); + return nullptr; } -String createCommaSeparatedStringFrom(const Strings & strings) + +ASTPtr extractOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.order_by) + return storage.order_by->clone(); + + throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); +} + + +String createCommaSeparatedStringFrom(const Names & strings) { String answer; for (auto & string: strings) @@ -159,20 +182,50 @@ String createCommaSeparatedStringFrom(const Strings & strings) return answer; } -Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) { - const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + const auto sorting_key_ast = extractOrderBy(storage_ast); + const auto primary_key_ast = extractPrimaryKey(storage_ast); - ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); - ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + const auto sorting_key_expr_list = MergeTreeData::extractKeyExpressionList(sorting_key_ast); + const auto primary_key_expr_list = primary_key_ast + ? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone(); - Strings answer; - answer.reserve(primary_key_or_order_by_arguments.size()); + /// VersionedCollapsing ??? - for (auto & column : primary_key_or_order_by_arguments) - answer.push_back(column->getColumnName()); + size_t primary_key_size = primary_key_expr_list->children.size(); + size_t sorting_key_size = sorting_key_expr_list->children.size(); - return answer; + if (primary_key_size > sorting_key_size) + throw Exception("Primary key must be a prefix of the sorting key, but its length: " + + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size), + ErrorCodes::BAD_ARGUMENTS); + + Names primary_key_columns; + Names sorting_key_columns; + NameSet primary_key_columns_set; + + for (size_t i = 0; i < sorting_key_size; ++i) + { + String sorting_key_column = sorting_key_expr_list->children[i]->getColumnName(); + sorting_key_columns.push_back(sorting_key_column); + + if (i < primary_key_size) + { + String pk_column = primary_key_expr_list->children[i]->getColumnName(); + if (pk_column != sorting_key_column) + throw Exception("Primary key must be a prefix of the sorting key, but in position " + + toString(i) + " its column is " + pk_column + ", not " + sorting_key_column, + ErrorCodes::BAD_ARGUMENTS); + + if (!primary_key_columns_set.emplace(pk_column).second) + throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS); + + primary_key_columns.push_back(pk_column); + } + } + + return primary_key_columns; } ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4f0db4a1cc3..ca20f88d870 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -202,11 +202,13 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast); * the data is sorted by both the old and new sorting keys at the moment of table modification. * * */ -ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast); +ASTPtr extractPrimaryKey(const ASTPtr & storage_ast); -String createCommaSeparatedStringFrom(const Strings & strings); +ASTPtr extractOrderBy(const ASTPtr & storage_ast); -Strings extractPrimaryKeyString(const ASTPtr & storage_ast); +String createCommaSeparatedStringFrom(const Names & strings); + +Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 47c6fbed948..e3a6a7b1bb1 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -254,7 +254,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); std::cout << engine_push_str << std::endl; - primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); From d9765991a7c0c94f8485e3040c17e17d11b71a59 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 16:15:23 +0300 Subject: [PATCH 026/147] replicated engine support --- dbms/programs/copier/ClusterCopier.cpp | 24 ++++++++-------- dbms/programs/copier/Internals.cpp | 29 ++++++++++++++++++-- dbms/programs/copier/Internals.h | 2 ++ dbms/programs/copier/TaskTableAndShard.h | 35 ++++++++++++++++++++++-- 4 files changed, 74 insertions(+), 16 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index e3013c5bebe..1a7ddce7e5e 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -480,10 +480,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - std::cout << piece_is_dirty_flag_path << std::endl; - std::cout << piece_is_dirty_cleaned_path << std::endl; - std::cout << piece_task_status_path << std::endl; - const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); @@ -559,7 +555,8 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast } /// Replaces ENGINE and table name in a create query -std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +std::shared_ptr ClusterCopier::rewriteCreateQueryStorage( + const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -574,6 +571,7 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A res->set(res->columns_list, create.columns_list->clone()); res->set(res->storage, new_storage_ast->clone()); + return res; } @@ -996,10 +994,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( CleanStateClock clean_state_clock(zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - std::cout << piece_is_dirty_flag_path << std::endl; - std::cout << piece_is_dirty_cleaned_path << std::endl; - std::cout << piece_status_path << std::endl; - const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); /// Do not start if partition piece is dirty, try to clean it @@ -1142,8 +1136,16 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( task_table.table_push.first, task_table.table_push.second + "_piece_" + toString(current_piece_number)); - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - database_and_table_for_current_piece, task_table.engine_push_ast); + auto new_engine_push_ast = task_table.engine_push_ast; + if (task_table.isReplicatedTable()) + { + new_engine_push_ast = task_table.rewriteParamsForReplicatedTableFor(current_piece_number); + } + + auto create_query_push_ast = rewriteCreateQueryStorage( + task_shard.current_pull_table_create_query, + database_and_table_for_current_piece, new_engine_push_ast); + create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index a0beccbde2c..263bf5f8590 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -78,8 +78,6 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - std::cout << "inside extractPartitionKey " << storage_str << std::endl; - const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); @@ -228,6 +226,33 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) return primary_key_columns; } +String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception( + "Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (!startsWith(engine.name, "Replicated")) + { + return ""; + } + + auto replicated_table_arguments = engine.arguments->children; + + auto zk_table_path_ast = replicated_table_arguments[0]->as(); + auto zk_table_path_string = zk_table_path_ast.value.safeGet(); + + return zk_table_path_string; +} + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { ShardPriority res; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index ca20f88d870..14dc888a0b4 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -210,6 +210,8 @@ String createCommaSeparatedStringFrom(const Names & strings); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); +String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast); + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); } diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index e3a6a7b1bb1..b9f90dc1a77 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -39,6 +39,11 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; + String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; + + + bool isReplicatedTable() { return engine_push_zk_path != ""; } + /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) size_t number_of_splits; @@ -65,6 +70,11 @@ struct TaskTable { ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; + /// First argument of Replicated...MergeTree() + String engine_push_zk_path; + + ASTPtr rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const; + /* * A Distributed table definition used to split data * Distributed table will be created on each shard of default @@ -251,10 +261,9 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf { ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - std::cout << engine_push_str << std::endl; primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); + engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); } sharding_key_str = config.getString(table_prefix + "sharding_key"); @@ -346,6 +355,24 @@ inline void TaskTable::initShards(RandomEngine && random_engine) local_shards.assign(all_shards.begin(), it_first_remote); } +inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const +{ + assert (engine_push_zk_path != ""); + return engine_push_zk_path + "/" + toString(piece_number); +} + +inline ASTPtr TaskTable::rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const +{ + const auto & new_engine_ast = engine_push_ast->clone()->as(); + + auto & replicated_table_arguments = new_engine_ast.arguments->children; + + auto & zk_table_path_ast = replicated_table_arguments[0]->as(); + zk_table_path_ast.value = getReplicatedEngineFirstArgumentForCurrentPiece(current_piece_number); + + return new_engine_ast.clone(); +} + inline String DB::TaskShard::getDescription() const { @@ -359,8 +386,10 @@ inline String DB::TaskShard::getDescription() const inline String DB::TaskShard::getHostNameExample() const { - auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); return replicas.at(0).readableString(); } + + } From 649df01627c2e9e388b0d21a5119072080ed53da Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 19:36:47 +0300 Subject: [PATCH 027/147] rewrite replicated tables to plain merge tree --- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/programs/copier/TaskTableAndShard.h | 23 +++++++++++++++-------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 1a7ddce7e5e..cdf530e8437 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1139,7 +1139,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto new_engine_push_ast = task_table.engine_push_ast; if (task_table.isReplicatedTable()) { - new_engine_push_ast = task_table.rewriteParamsForReplicatedTableFor(current_piece_number); + new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain(); } auto create_query_push_ast = rewriteCreateQueryStorage( diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index b9f90dc1a77..c879152d6c9 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -39,7 +39,7 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; + [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; bool isReplicatedTable() { return engine_push_zk_path != ""; } @@ -73,7 +73,7 @@ struct TaskTable { /// First argument of Replicated...MergeTree() String engine_push_zk_path; - ASTPtr rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const; + ASTPtr rewriteReplicatedCreateQueryToPlain(); /* * A Distributed table definition used to split data @@ -358,19 +358,26 @@ inline void TaskTable::initShards(RandomEngine && random_engine) inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const { assert (engine_push_zk_path != ""); - return engine_push_zk_path + "/" + toString(piece_number); + return engine_push_zk_path + "/piece_" + toString(piece_number); } -inline ASTPtr TaskTable::rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const +inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() { - const auto & new_engine_ast = engine_push_ast->clone()->as(); + ASTPtr prev_engine_push_ast = engine_push_ast->clone(); + + auto & new_storage_ast = prev_engine_push_ast->as(); + auto & new_engine_ast = new_storage_ast.engine->as(); auto & replicated_table_arguments = new_engine_ast.arguments->children; - auto & zk_table_path_ast = replicated_table_arguments[0]->as(); - zk_table_path_ast.value = getReplicatedEngineFirstArgumentForCurrentPiece(current_piece_number); + /// Delete first two arguments of Replicated...MergeTree() table. + replicated_table_arguments.erase(replicated_table_arguments.begin()); + replicated_table_arguments.erase(replicated_table_arguments.begin()); - return new_engine_ast.clone(); + /// Remove replicated from name + new_engine_ast.name = new_engine_ast.name.substr(10); + + return new_storage_ast.clone(); } From 12d5900d71d29a26189c9780f457a570ea927a0d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 7 Mar 2020 03:05:49 +0300 Subject: [PATCH 028/147] most tests ok --- dbms/programs/copier/ClusterCopier.cpp | 62 ++++++++++++++++--- .../Storages/StorageReplicatedMergeTree.cpp | 3 + .../integration/test_cluster_copier/test.py | 10 +++ 3 files changed, 65 insertions(+), 10 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index cdf530e8437..6eb0ffcfac9 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -904,17 +904,43 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - PartitionTaskStatus res; - PartitionTaskStatus answer = PartitionTaskStatus::Finished; + PartitionTaskStatus res{PartitionTaskStatus::Finished}; + + bool was_failed_pieces = false; + bool was_active_pieces = false; + bool was_error = false; for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) { - res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - if (res == PartitionTaskStatus::Error) - answer = res; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + /// Exit if success + if (res == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (res == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + was_active_pieces = (res == PartitionTaskStatus::Active); + was_failed_pieces = (res == PartitionTaskStatus::Error); } - return answer; + if (was_failed_pieces) + return PartitionTaskStatus::Error; + + if (was_active_pieces) + return PartitionTaskStatus::Active; + + return PartitionTaskStatus::Finished; } @@ -1266,6 +1292,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// Try create original table (if not exists) on each shard + try { auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); @@ -1279,6 +1306,10 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } + catch (...) + { + tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); + } /// Move partition to original destination table. { @@ -1293,16 +1324,25 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(helping_table) + - " MOVE PARTITION " + task_partition.name + - " TO TABLE " + getQuotedTable(original_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + task_partition.name + + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); +// query_alter_ast_string += " INSERT INTO " + getQuotedTable(original_table) + +// " SELECT * FROM " + getQuotedTable(helping_table); +// +// query_alter_ast_string += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; +// +// LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + try { - UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_ONE, 1); + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_MANY); + assert(num_shards > 0); LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); } catch (...) @@ -1436,6 +1476,8 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout task_shard.list_of_split_tables_on_shard[piece_number], storage_piece_split_ast); + std::cout << "anime" << queryToString(create_table_split_piece_ast) << std::endl; + dropAndCreateLocalTable(create_table_split_piece_ast); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0ed2527a981..7ea195d496d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4877,6 +4877,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ ErrorCodes::LOGICAL_ERROR); String hash_hex = src_part->checksums.getTotalChecksumHex(); + + LOG_INFO(log, "Trying to attach " << src_part->name << "with hash_hex " << hash_hex); + String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index c223a73f59e..4677c49eb43 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -92,6 +92,16 @@ class Task1: def check(self): assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = self.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print(self.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) + + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") + assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") From 0df165578bf0f5b9b08040b21356c8c88a089692 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 10 Mar 2020 17:42:06 +0300 Subject: [PATCH 029/147] mark piece as done if it is not present --- dbms/programs/copier/ClusterCopier.cpp | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 6eb0ffcfac9..45b386a7503 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1065,6 +1065,26 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } + + /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check + /// whether each shard have processed each partitition (and its pieces). + if (partition_piece.is_absent_piece) + { + std::cout << "current partition piece is clean?? " << is_clean << std::endl; + std::cout << "######" << "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " IS ABSENT ON CURRENT SHARD" << std::endl; + std::cout << "current_task_piece_status_path " << current_task_piece_status_path << std::endl; + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); + if (res == Coordination::ZNODEEXISTS) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. But other replica has already marked it as done."); + if (res == Coordination::ZOK) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); + return PartitionTaskStatus::Finished; + } + /// Exit if task has been already processed; /// create blocking node to signal cleaning up if it is abandoned { @@ -1376,8 +1396,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// TODO: LOG_INFO (Piece copied and moved to destination table) - - return PartitionTaskStatus::Finished; } From f93aaf061fbb56e3e03615e6ecb9c9507fd6ad18 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 10 Mar 2020 23:04:08 +0300 Subject: [PATCH 030/147] probably all tests are ok --- dbms/programs/copier/ClusterCopier.cpp | 101 ++++++++++++------ dbms/programs/copier/ClusterCopier.h | 13 +++ dbms/programs/copier/ShardPartitionPiece.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 2 +- dbms/src/Interpreters/Cluster.h | 12 +++ .../Storages/StorageReplicatedMergeTree.cpp | 6 +- .../integration/test_cluster_copier/test.py | 19 ++-- 7 files changed, 112 insertions(+), 45 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 45b386a7503..f6f3d2bed55 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -674,7 +674,13 @@ bool ClusterCopier::tryDropPartitionPiece( LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + UInt64 num_shards = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_ONE, + ClusterExecutionMode::ON_EACH_SHARD, + 1); if (num_shards < cluster_push->getShardCount()) { @@ -1065,26 +1071,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } - - /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check - /// whether each shard have processed each partitition (and its pieces). - if (partition_piece.is_absent_piece) - { - std::cout << "current partition piece is clean?? " << is_clean << std::endl; - std::cout << "######" << "Partition " << task_partition.name - << " piece " + toString(current_piece_number) + " IS ABSENT ON CURRENT SHARD" << std::endl; - std::cout << "current_task_piece_status_path " << current_task_piece_status_path << std::endl; - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); - if (res == Coordination::ZNODEEXISTS) - LOG_DEBUG(log, "Partition " << task_partition.name << " piece " - + toString(current_piece_number) + " is absent on current replica of a shard. But other replica has already marked it as done."); - if (res == Coordination::ZOK) - LOG_DEBUG(log, "Partition " << task_partition.name << " piece " - + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); - return PartitionTaskStatus::Finished; - } - /// Exit if task has been already processed; /// create blocking node to signal cleaning up if it is abandoned { @@ -1110,6 +1096,22 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } } + + /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check + /// whether each shard have processed each partitition (and its pieces). + if (partition_piece.is_absent_piece) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); + if (res == Coordination::ZNODEEXISTS) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. But other replicas have already marked it as done."); + if (res == Coordination::ZOK) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); + return PartitionTaskStatus::Finished; + } + /// Check that destination partition is empty if we are first worker /// NOTE: this check is incorrect if pull and push tables have different partition key! String clean_start_status; @@ -1119,7 +1121,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - /// TODO: Why table_split_shard??? + ///TODO: Why table_split_shard??? ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; @@ -1350,20 +1352,20 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); -// query_alter_ast_string += " INSERT INTO " + getQuotedTable(original_table) + -// " SELECT * FROM " + getQuotedTable(helping_table); -// -// query_alter_ast_string += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; -// -// LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - try { ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_MANY); + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - assert(num_shards > 0); - LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); + // TODO: Throw an exception if num nodes is not equal to original number of nodes + + LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) { @@ -1623,6 +1625,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( const ASTPtr & query_ast_, const Settings * settings, PoolMode pool_mode, + ClusterExecutionMode execution_mode, UInt64 max_successful_executions_per_shard) const { auto num_shards = cluster->getShardsInfo().size(); @@ -1637,6 +1640,11 @@ UInt64 ClusterCopier::executeQueryOnCluster( else query_ast = query_ast_; + /// We will have to execute query on each replica of a shard. + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) + max_successful_executions_per_shard = 0; + + std::atomic origin_replicas_number; /// We need to execute query on one replica at least auto do_for_shard = [&] (UInt64 shard_index) @@ -1652,6 +1660,13 @@ UInt64 ClusterCopier::executeQueryOnCluster( }; UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + + for (size_t i = 0; i < num_replicas; ++i) + { + std::cout << "host_name " << cluster->getShardsAddresses().at(shard_index)[i].host_name + << " port " << cluster->getShardsAddresses().at(shard_index)[i].port << std::endl; + } + origin_replicas_number += num_replicas; UInt64 num_local_replicas = shard.getLocalNodeCount(); UInt64 num_remote_replicas = num_replicas - num_local_replicas; @@ -1706,10 +1721,26 @@ UInt64 ClusterCopier::executeQueryOnCluster( thread_pool.wait(); } - UInt64 successful_shards = 0; + UInt64 successful_nodes = 0; for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); + { + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) + successful_nodes += num_replicas; + else + /// Count only successful shards + successful_nodes += (num_replicas > 0); + } - return successful_shards; + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number - 1) + { + LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on " + << toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load())); + + std::cout << "successful_nodes " << successful_nodes << " origin_replicas_number " << origin_replicas_number << std::endl; + throw Exception("There was an error while executing ALTER on each node.", ErrorCodes::LOGICAL_ERROR); + } + + + return successful_nodes; } } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index a5587045299..4bbf9e58286 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -149,6 +149,18 @@ protected: bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); + /* + * This class is used in executeQueryOnCluster function + * You can execute query on each shard (no sense it is executed on each replica of a shard or not) + * or you can execute query on each replica on each shard. + * First mode is useful for INSERTS queries. + * */ + enum ClusterExecutionMode + { + ON_EACH_SHARD, + ON_EACH_NODE + }; + /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully */ @@ -158,6 +170,7 @@ protected: const ASTPtr & query_ast_ = nullptr, const Settings * settings = nullptr, PoolMode pool_mode = PoolMode::GET_ALL, + ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, UInt64 max_successful_executions_per_shard = 0) const; private: diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 04f7f458a00..88f1ceddb0d 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -6,8 +6,8 @@ namespace DB { struct ShardPartitionPiece { - ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) - : is_absent_piece(is_absent_piece_), current_piece_number(current_piece_number_), + ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_present_piece_) + : is_absent_piece(!is_present_piece_), current_piece_number(current_piece_number_), shard_partition(parent) {} [[maybe_unused]] String getPartitionPiecePath() const; diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index c879152d6c9..91e14598e81 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -241,7 +241,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf name_in_config = table_key; - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 2); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index eba046994eb..b6c8b054495 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -151,6 +151,18 @@ public: /// The number of all shards. size_t getShardCount() const { return shards_info.size(); } +// /// The number of all nodes (all replicas of each shard). +// size_t getNodesCount() const +// { +// size_t nodes_count = 0; +// std::cout << "addresses_with_failover.size() " << addresses_with_failover.size() << std::endl; +// std::for_each(addresses_with_failover.begin(), addresses_with_failover.end(), [&] (const Addresses & addresses) +// { +// nodes_count += addresses.size(); +// }); +// return nodes_count; +// } + /// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster. std::unique_ptr getClusterWithSingleShard(size_t index) const; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 9880df6f116..99cc73b56a6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4888,7 +4888,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ String hash_hex = src_part->checksums.getTotalChecksumHex(); - LOG_INFO(log, "Trying to attach " << src_part->name << "with hash_hex " << hash_hex); + LOG_INFO(log, "Trying to attach " << src_part->name << " with hash_hex " << hash_hex); String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); @@ -4910,6 +4910,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } + /// We have nothing to do - return + if (src_parts.empty()) + return; + ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 4677c49eb43..b363c02d71d 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,6 +68,13 @@ def started_cluster(): pass cluster.shutdown() +def print_destination_cluster(task): + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = task.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print(task.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) class Task1: @@ -91,13 +98,8 @@ class Task1: def check(self): + print_destination_cluster(self) assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = self.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) - print(self.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") @@ -136,6 +138,7 @@ class Task2: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -171,6 +174,8 @@ class Task_test_block_size: def check(self): + #print_destination_cluster(self) + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) instance = cluster.instances['s0_0_0'] @@ -194,6 +199,7 @@ class Task_no_index: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -217,6 +223,7 @@ class Task_no_arg: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From acf09b6630489cf34f9a92330735fafa942a2846 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 11 Mar 2020 16:01:17 +0300 Subject: [PATCH 031/147] fix build --- dbms/programs/copier/ClusterCopier.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f6f3d2bed55..27f2bb53e09 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -914,7 +914,6 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn bool was_failed_pieces = false; bool was_active_pieces = false; - bool was_error = false; for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) { @@ -926,8 +925,6 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn if (res == PartitionTaskStatus::Finished) break; - was_error = true; - /// Skip if the task is being processed by someone if (res == PartitionTaskStatus::Active) break; From a8611da113107409c13a3def9a3c882678b05b9b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 11 Mar 2020 22:55:27 +0300 Subject: [PATCH 032/147] maybe better --- dbms/programs/copier/ClusterCopier.cpp | 40 +++++++++++-------- dbms/programs/copier/ClusterCopier.h | 1 + dbms/src/Interpreters/Cluster.cpp | 4 ++ .../Storages/StorageReplicatedMergeTree.cpp | 2 +- .../integration/test_cluster_copier/test.py | 35 ++++++++++++---- 5 files changed, 58 insertions(+), 24 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 27f2bb53e09..d7a9a7922dc 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -43,6 +43,13 @@ void ClusterCopier::init() for (auto & task_table : task_cluster->table_tasks) { task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + auto pull_shards_info = task_table.cluster_pull->getShardsInfo(); + for (auto & shard_info : pull_shards_info) + { + std::cout << "current_shard " << toString(shard_info.shard_num) << "has remote connections " + << toString(shard_info.hasRemoteConnections()) << std::endl; + } + std::cout << "CLUSTER PULL " << std::endl; task_table.cluster_push = context.getCluster(task_table.cluster_push_name); task_table.initShards(task_cluster->random_engine); } @@ -1212,7 +1219,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ inject_fault ? "1" : ""); + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : ""); LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() << " : " << queryToString(query_select_ast)); @@ -1345,23 +1352,27 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + " ATTACH PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table); + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); try { - ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - // TODO: Throw an exception if num nodes is not equal to original number of nodes + size_t num_nodes = 0; + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) + { + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + num_nodes = std::max(current_num_nodes, num_nodes); + } LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) @@ -1728,13 +1739,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( successful_nodes += (num_replicas > 0); } - if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number - 1) + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number) { LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on " << toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load())); - - std::cout << "successful_nodes " << successful_nodes << " origin_replicas_number " << origin_replicas_number << std::endl; - throw Exception("There was an error while executing ALTER on each node.", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 4bbf9e58286..bed6352e129 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -116,6 +116,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 5; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 3261c0caf59..446b258d30f 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -75,6 +75,8 @@ bool Cluster::Address::isLocal(UInt16 clickhouse_port) const Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_, UInt32 replica_index_) : shard_index(shard_index_), replica_index(replica_index_) { + + std::cout << "FROM Address constructor " << " shard index " << shard_index_ << " replica index " << replica_index_ << std::endl; host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); if (config.has(config_prefix + ".user")) @@ -327,6 +329,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting if (startsWith(replica_key, "replica")) { replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num); + + std::cout << "replica num " << current_replica_num << " is a replica of shard number " << current_shard_num << std::endl; ++current_replica_num; if (!replica_addresses.back().is_local) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 99cc73b56a6..244dca2a044 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4828,7 +4828,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(true, context.getCurrentQueryId()); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); Stopwatch watch; diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index b363c02d71d..7af58c66907 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,13 +68,32 @@ def started_cluster(): pass cluster.shutdown() -def print_destination_cluster(task): +def print_destination_cluster(task, pattern): for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") c = task.cluster.instances[anime].query("SELECT count() FROM hits") print(anime, a, b, int(a) + int(b), c) - print(task.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) + print(task.cluster.instances[anime].query("select partition, " + "name, database, table, " + "rows, hash_of_all_files, " + "hash_of_uncompressed_files, " + "uncompressed_hash_of_compressed_files " + "from system.parts " + "where database='default'" + "format PrettyCompact".format(pattern))) + + +def print_source_cluster(task, pattern): + for anime in ['s0_0_0', 's0_0_1', 's0_1_0']: + print(task.cluster.instances[anime].query("select partition, " + "name, database, table, " + "rows, min_date, min_time, hash_of_all_files, " + "hash_of_uncompressed_files, " + "uncompressed_hash_of_compressed_files " + "from system.parts " + "where table like '%{}%' " + "format PrettyCompact".format(pattern))) class Task1: @@ -98,7 +117,7 @@ class Task1: def check(self): - print_destination_cluster(self) + print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") @@ -136,9 +155,11 @@ class Task2: instance.query("INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85", settings={"insert_distributed_sync": 1}) + print_source_cluster(self, "a") + def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "a") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -174,7 +195,7 @@ class Task_test_block_size: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "test_block_size") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) @@ -199,7 +220,7 @@ class Task_no_index: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "ontime") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -223,7 +244,7 @@ class Task_no_arg: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "copier_test1") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From c2033d2f0183fedc55e647dcd05eb728e16d7b0e Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 12 Mar 2020 14:51:59 +0300 Subject: [PATCH 033/147] Fix build errors. --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 90d1cf99b2e..f20ae2b6cfa 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -80,8 +80,6 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) else type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); break; - default: - __builtin_unreachable(); } } catch (Exception & e) From feea427267b5bd041ebe339fe299116f8fe14b76 Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 12 Mar 2020 17:18:39 +0300 Subject: [PATCH 034/147] Fix build errors 2 --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index f20ae2b6cfa..e26255deae4 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -1,9 +1,9 @@ +#include #include #include #include #include #include -#include namespace DB { From 12dd0c92c09b2ad022ada4fe62a55e30e1806ad3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 19:48:28 +0300 Subject: [PATCH 035/147] better --- dbms/programs/copier/ClusterCopier.cpp | 56 +++++++++++++++---- dbms/programs/copier/ZooKeeperStaff.h | 3 + .../integration/test_cluster_copier/test.py | 10 ++-- 3 files changed, 55 insertions(+), 14 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d7a9a7922dc..8359db65e28 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -414,6 +414,9 @@ bool ClusterCopier::checkPartitionPieceIsClean( task_start_clock = LogicalClock(stat.mzxid); /// If statement for readability. + + LOG_INFO(log, "clean_state_clock.is_stale() " << clean_state_clock.is_stale()); + LOG_INFO(log, "clean_state_clock.is_clean() " << clean_state_clock.is_clean()); if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { return true; @@ -680,20 +683,20 @@ bool ClusterCopier::tryDropPartitionPiece( settings_push.replication_alter_partitions_sync = 2; LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 + /// We have to drop partition_piece on each replica UInt64 num_shards = executeQueryOnCluster( cluster_push, query, nullptr, &settings_push, - PoolMode::GET_ONE, - ClusterExecutionMode::ON_EACH_SHARD, - 1); + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } + UNUSED(num_shards); +// if (num_shards < cluster_push->getShardCount()) +// { +// LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); +// return false; +// } /// Update the locking node if (!my_clock.is_stale()) @@ -717,7 +720,7 @@ bool ClusterCopier::tryDropPartitionPiece( zookeeper->set(current_shards_path, host_id); } - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is safe for work now."); return true; } @@ -926,6 +929,9 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) { + LOG_INFO(log, "Attempt number " << try_num << " to process partition " << task_partition.name + << " piece number " << piece_number << " on shard number " << task_partition.task_shard.numberInCluster() + << " with index " << task_partition.task_shard.indexInCluster()); res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); /// Exit if success @@ -1141,6 +1147,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (count != 0) { + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << current_piece_number << "is not empty. In contains " << count << " rows."); Coordination::Stat stat_shards{}; zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards); @@ -1309,6 +1317,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( catch (...) { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } @@ -1382,6 +1391,33 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } + + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) + { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + task_partition.name + " DEDUPLICATE;"; + + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); + } + } + catch(...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << task_partition.name << "in the original table"); + throw; + } + } /// Finalize the processing, change state of current partition task (and also check is_dirty flag) diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index 2fc4d35400d..e22df6bf199 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,6 +157,9 @@ public: bool is_clean() const { + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "!discovery_zxid.hasHappened() " << !discovery_zxid.hasHappened()); + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "clean_state_zxid.hasHappened() " << clean_state_zxid.hasHappened()); + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "discovery_zxid <= clean_state_zxid " << (discovery_zxid <= clean_state_zxid)); return !is_stale() && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 7af58c66907..9ea26013125 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -70,10 +70,6 @@ def started_cluster(): def print_destination_cluster(task, pattern): for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = task.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) print(task.cluster.instances[anime].query("select partition, " "name, database, table, " "rows, hash_of_all_files, " @@ -117,6 +113,12 @@ class Task1: def check(self): + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = self.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") From fc599780b653c6a57e0c4bc88fc49ed8ec6c6851 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:46:48 +0300 Subject: [PATCH 036/147] slightly better code --- dbms/programs/copier/ClusterCopier.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 8359db65e28..62c51952e95 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -152,14 +152,13 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, for (const String & partition_name : filtered_partitions_names) { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + const size_t number_of_splits = task_table.number_of_splits; + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, number_of_splits)); task_shard->checked_partitions.emplace(partition_name, true); auto shard_partition_it = task_shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - const size_t number_of_splits = task_table.number_of_splits; - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); @@ -670,7 +669,11 @@ bool ClusterCopier::tryDropPartitionPiece( } } - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push) + "_piece_" + toString(current_piece_number); + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "ALTER TABLE " + getQuotedTable(helping_table); query += " DROP PARTITION " + task_partition.name + ""; /// TODO: use this statement after servers will be updated up to 1.1.54310 @@ -764,14 +767,13 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab if (has_partition) { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + const size_t number_of_splits = task_table.number_of_splits; + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits)); LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); /// To save references in the future. auto shard_partition_it = shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - const size_t number_of_splits = task_table.number_of_splits; - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); From 65d6cc0aa98f1f513c411be6a1f5fe10712e0c52 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:52:36 +0300 Subject: [PATCH 037/147] codestyle --- dbms/programs/copier/Internals.h | 1 - dbms/programs/copier/TaskTableAndShard.h | 22 +++++++++++++--------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4e4142a035b..fc4c2f09da7 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 91e14598e81..b33876c679e 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -286,7 +286,8 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf } where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) { + if (!where_condition_str.empty()) + { ParserExpressionWithOptionalAlias parser_expression(false); where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); @@ -297,28 +298,31 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf String enabled_partitions_prefix = table_prefix + "enabled_partitions"; has_enabled_partitions = config.has(enabled_partitions_prefix); - if (has_enabled_partitions) { + if (has_enabled_partitions) + { Strings keys; config.keys(enabled_partitions_prefix, keys); - if (keys.empty()) { + if (keys.empty()) + { /// Parse list of partition from space-separated string String partitions_str = config.getString(table_prefix + "enabled_partitions"); boost::trim_if(partitions_str, isWhitespaceASCII); boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } else { + } + else + { /// Parse sequence of ... - for (const String &key : keys) { + for (const String &key : keys) + { if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); } } - std::copy(enabled_partitions.begin(), enabled_partitions.end(), - std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); } } From 88fc6f2e91d3dd65590b34d5427d7c965bc0799e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:54:37 +0300 Subject: [PATCH 038/147] remove unused code --- dbms/src/Parsers/ASTLiteral.h | 4 ++-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ---- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 1d307a4101e..552f5da04a2 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -21,8 +21,8 @@ public: std::optional begin; std::optional end; - explicit ASTLiteral(Field && value_) : value(value_) {} - explicit ASTLiteral(const Field & value_) : value(value_) {} + ASTLiteral(Field && value_) : value(value_) {} + ASTLiteral(const Field & value_) : value(value_) {} /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 244dca2a044..891e3beb0c1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4910,10 +4910,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// We have nothing to do - return - if (src_parts.empty()) - return; - ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From ad7a9314903b428c2ca6c778f37085200113288d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 23:13:31 +0300 Subject: [PATCH 039/147] return back fixes in StorageReplicatedMergeTree --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 891e3beb0c1..244dca2a044 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4910,6 +4910,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } + /// We have nothing to do - return + if (src_parts.empty()) + return; + ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From cce69b0744bd337bbacf8abbc32d1f2ae604ebfb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 13:14:48 +0300 Subject: [PATCH 040/147] fix mac build --- dbms/programs/copier/ClusterCopier.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 62c51952e95..f308a9a55f5 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -2,6 +2,8 @@ #include "Internals.h" +#include + #include #include From 42fae556c11de775760ec946832bd5a9adfa911c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 17:19:20 +0300 Subject: [PATCH 041/147] split up copying and moving --- dbms/programs/copier/ClusterCopier.cpp | 259 ++++++++++++++++------- dbms/programs/copier/ClusterCopier.h | 4 + dbms/programs/copier/TaskTableAndShard.h | 21 +- 3 files changed, 199 insertions(+), 85 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f308a9a55f5..d7be72022f1 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -433,8 +433,14 @@ bool ClusterCopier::checkPartitionPieceIsClean( bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) { bool answer = true; - for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) - answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; ++piece_number) + { + bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + if (!piece_is_done) + LOG_DEBUG(log, "Partition " << partition_name << " piece " + toString(piece_number) + " is not already done."); + answer &= piece_is_done; + } + return answer; } @@ -528,10 +534,156 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } } - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " is copied successfully"); return true; } + +PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) +{ + LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table"); + + auto zookeeper = context.getZooKeeper(); + + const auto current_partition_attach_is_active = task_table.getPartitionAttachIsActivePath(partition_name); + const auto current_partition_attach_is_done = task_table.getPartitionAttachIsDonePath(partition_name); + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_partition_attach_is_active); + zkutil::EphemeralNodeHolderPtr partition_attach_node_holder; + try + { + partition_attach_node_holder = zkutil::EphemeralNodeHolder::create(current_partition_attach_is_active, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active); + return PartitionTaskStatus::Active; + } + + throw; + } + + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_partition_attach_is_done, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active + << " has been successfully moved to destination table by " << status.owner); + return PartitionTaskStatus::Finished; + } + + /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. + /// Initialize DROP PARTITION + LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active + << " has not been successfully finished by " << status.owner + << ". Will try to move by myself."); + } + } + + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); + } + + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + /// Move partition to original destination table. + { + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << partition_name + << " piece " << toString(current_piece_number) << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + partition_name + + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try + { + size_t num_nodes = 0; + + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) + { + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + num_nodes = std::max(current_num_nodes, num_nodes); + } + + LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << partition_name + << " piece " << toString(current_piece_number) << "to original table"); + throw; + } + + + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) + { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + partition_name + " DEDUPLICATE;"; + + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); + } + } + catch(...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name << "in the original table"); + throw; + } + } + } + + + /// Create node to signal that we finished moving + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + zookeeper->set(current_partition_attach_is_done, state_finished, 0); + } + + return PartitionTaskStatus::Finished; +} + /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { @@ -838,20 +990,37 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Check that whole cluster partition is done /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; + bool partition_copying_is_done = num_failed_shards == 0; try { - partition_is_done = + partition_copying_is_done = !has_shard_to_process - || (partition_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); + || (partition_copying_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); } catch (...) { tryLogCurrentException(log); - partition_is_done = false; + partition_copying_is_done = false; } - if (partition_is_done) + + bool partition_moving_is_done = false; + /// Try to move only if all pieces were copied. + if (partition_copying_is_done) + { + try + { + auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); + if (res == PartitionTaskStatus::Finished) + partition_moving_is_done = true; + } + catch (...) + { + tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); + } + } + + if (partition_copying_is_done && partition_moving_is_done) { task_table.finished_cluster_partitions.emplace(partition_name); @@ -1350,80 +1519,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); } - /// Move partition to original destination table. - { - /// TODO: Execute alter table move partition. - - LOG_DEBUG(log, "Trying to move partition " << task_partition.name - << " piece " << toString(current_piece_number) << " to original table"); - - ASTPtr query_alter_ast; - String query_alter_ast_string; - - DatabaseAndTableName original_table = task_table.table_push; - DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " ATTACH PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; - - LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - - try - { - size_t num_nodes = 0; - for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) - { - ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 current_num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - num_nodes = std::max(current_num_nodes, num_nodes); - } - LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); - } - catch (...) - { - LOG_DEBUG(log, "Error while moving partition " << task_partition.name - << " piece " << toString(current_piece_number) << "to original table"); - throw; - } - - - try - { - String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) - { - query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + - " PARTITION " + task_partition.name + " DEDUPLICATE;"; - - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); - - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_deduplicate_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY); - - LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); - } - } - catch(...) - { - LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << task_partition.name << "in the original table"); - throw; - } - - } - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index bed6352e129..a39653c6c87 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -104,6 +104,10 @@ protected: bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, size_t piece_number, const TasksShard & shards_with_partition); + + /*Alter successful insertion to helping tables it will move all pieces to destination table*/ + PartitionTaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); + /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index b33876c679e..9a09deb911f 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -25,6 +25,10 @@ struct TaskTable { String getPartitionPath(const String & partition_name) const; + String getPartitionAttachIsActivePath(const String & partition_name) const; + + String getPartitionAttachIsDonePath(const String & partition_name) const; + String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsDirtyPath(const String & partition_name) const; @@ -42,7 +46,7 @@ struct TaskTable { [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; - bool isReplicatedTable() { return engine_push_zk_path != ""; } + bool isReplicatedTable() const { return engine_push_zk_path != ""; } /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -191,12 +195,23 @@ struct TaskShard }; -inline String TaskTable::getPartitionPath(const String &partition_name) const { +inline String TaskTable::getPartitionPath(const String & partition_name) const +{ return task_cluster.task_zookeeper_path // root + "/tables/" + table_id // tables/dst_cluster.merge.hits + "/" + escapeForFileName(partition_name); // 201701 } +inline String TaskTable::getPartitionAttachIsActivePath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/attach_active"; +} + +inline String TaskTable::getPartitionAttachIsDonePath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/attach_is_done"; +} + inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { assert(piece_number < number_of_splits); @@ -241,7 +256,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf name_in_config = table_key; - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 2); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); From b26a949c372455e11b58e767645cf445a14a0b70 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:25:07 +0300 Subject: [PATCH 042/147] drop helping tables --- dbms/programs/copier/ClusterCopier.cpp | 30 ++++++++++++++++++++++++++ dbms/programs/copier/ClusterCopier.h | 2 ++ 2 files changed, 32 insertions(+) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d7be72022f1..471db73e715 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -318,6 +318,9 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) } } + /// Delete helping tables in both cases (whole table is done or not) + dropHelpingTables(task_table); + if (!table_is_done) { throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", @@ -1564,6 +1567,33 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na interpreter.execute(); } + +void ClusterCopier::dropHelpingTables(const TaskTable & task_table) +{ + LOG_DEBUG(log, "Removing helping tables"); + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "DROP TABLE IF EXISTS " + getQuotedTable(helping_table); + + const ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + LOG_DEBUG(log, "Execute distributed DROP TABLE: " << query); + /// We have to drop partition_piece on each replica + UInt64 num_nodes = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + LOG_DEBUG(log, "DROP TABLE query was successfully executed on " << toString(num_nodes) << " nodes."); + } +} + String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index a39653c6c87..ce9ea7cc7ef 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -137,6 +137,8 @@ protected: void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const; + void dropHelpingTables(const TaskTable & task_table); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); From 3e394bee2468e9cf2d77e97023f938c168c41eec Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:39:16 +0300 Subject: [PATCH 043/147] remove useless prints from test --- .../integration/test_cluster_copier/test.py | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 9ea26013125..47df976bbcf 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,28 +68,6 @@ def started_cluster(): pass cluster.shutdown() -def print_destination_cluster(task, pattern): - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - print(task.cluster.instances[anime].query("select partition, " - "name, database, table, " - "rows, hash_of_all_files, " - "hash_of_uncompressed_files, " - "uncompressed_hash_of_compressed_files " - "from system.parts " - "where database='default'" - "format PrettyCompact".format(pattern))) - - -def print_source_cluster(task, pattern): - for anime in ['s0_0_0', 's0_0_1', 's0_1_0']: - print(task.cluster.instances[anime].query("select partition, " - "name, database, table, " - "rows, min_date, min_time, hash_of_all_files, " - "hash_of_uncompressed_files, " - "uncompressed_hash_of_compressed_files " - "from system.parts " - "where table like '%{}%' " - "format PrettyCompact".format(pattern))) class Task1: @@ -113,18 +91,7 @@ class Task1: def check(self): - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = self.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) - - print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") - - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") - assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") @@ -157,11 +124,7 @@ class Task2: instance.query("INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85", settings={"insert_distributed_sync": 1}) - print_source_cluster(self, "a") - - def check(self): - print_destination_cluster(self, "a") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -197,8 +160,6 @@ class Task_test_block_size: def check(self): - print_destination_cluster(self, "test_block_size") - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) instance = cluster.instances['s0_0_0'] @@ -222,7 +183,6 @@ class Task_no_index: def check(self): - print_destination_cluster(self, "ontime") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -246,7 +206,6 @@ class Task_no_arg: def check(self): - print_destination_cluster(self, "copier_test1") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From b6d633a09ef7ba812dd95c3e09b273b00d84273b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:53:32 +0300 Subject: [PATCH 044/147] more comments --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 244dca2a044..c37623986c3 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4888,7 +4888,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ String hash_hex = src_part->checksums.getTotalChecksumHex(); - LOG_INFO(log, "Trying to attach " << src_part->name << " with hash_hex " << hash_hex); + LOG_INFO(log, "Trying to " << (replace ? "replace " : "attach ") << src_part->name << " with hash_hex " << hash_hex); String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); @@ -4910,8 +4910,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// We have nothing to do - return - if (src_parts.empty()) + /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. + /// In case of REPLACE PARTITION we can replace existing partition with empty. + if (!replace && src_parts.empty()) return; ReplicatedMergeTreeLogEntryData entry; From 91f57e46e9e93497b3249b33a6aa8a5c6ad9def9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 20:05:05 +0300 Subject: [PATCH 045/147] fix mac build --- dbms/programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 471db73e715..88590205850 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -628,7 +628,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) { ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 current_num_nodes = executeQueryOnCluster( + size_t current_num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, nullptr, From 25ba98d0aec23e2041d102333084d2ac3cecc7c2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Mar 2020 11:25:59 +0300 Subject: [PATCH 046/147] get rid of logs --- dbms/programs/copier/ZooKeeperStaff.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index e22df6bf199..2fc4d35400d 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,9 +157,6 @@ public: bool is_clean() const { - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "!discovery_zxid.hasHappened() " << !discovery_zxid.hasHappened()); - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "clean_state_zxid.hasHappened() " << clean_state_zxid.hasHappened()); - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "discovery_zxid <= clean_state_zxid " << (discovery_zxid <= clean_state_zxid)); return !is_stale() && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } From d2179339488f682dae6200ff5403a6ee2bf7c9fb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 00:05:38 +0300 Subject: [PATCH 047/147] add more tests --- dbms/programs/copier/ClusterCopier.cpp | 169 ++++++++++-------- dbms/programs/copier/ClusterCopier.h | 8 +- dbms/programs/copier/ClusterCopierApp.cpp | 5 + dbms/programs/copier/ClusterCopierApp.h | 3 +- .../configs/conf.d/clusters.xml | 19 ++ .../test_cluster_copier/task_trivial.xml | 64 +++++++ .../integration/test_cluster_copier/test.py | 13 +- .../test_cluster_copier/trivial_test.py | 144 +++++++++++++++ 8 files changed, 348 insertions(+), 77 deletions(-) create mode 100644 dbms/tests/integration/test_cluster_copier/task_trivial.xml create mode 100644 dbms/tests/integration/test_cluster_copier/trivial_test.py diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 88590205850..b1c284122a4 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -544,6 +544,13 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) { + bool inject_fault = false; + if (move_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < move_fault_probability; + } + LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table"); auto zookeeper = context.getZooKeeper(); @@ -599,85 +606,90 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); } + /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - /// Move partition to original destination table. + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << partition_name + << " piece " << toString(current_piece_number) + << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, + original_table.second + "_piece_" + + toString(current_piece_number)); + + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + partition_name + + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try { - /// TODO: Execute alter table move partition. + size_t num_nodes = 0; - LOG_DEBUG(log, "Trying to move partition " << partition_name - << " piece " << toString(current_piece_number) << " to original table"); + ///FIXME: We have to be sure that every node in cluster executed this query + size_t current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - ASTPtr query_alter_ast; - String query_alter_ast_string; - - DatabaseAndTableName original_table = task_table.table_push; - DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " ATTACH PARTITION " + partition_name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; - - LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - - try - { - size_t num_nodes = 0; - - for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) - { - ///FIXME: We have to be sure that every node in cluster executed this query - size_t current_num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - num_nodes = std::max(current_num_nodes, num_nodes); - } - - LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); - } - catch (...) - { - LOG_DEBUG(log, "Error while moving partition " << partition_name - << " piece " << toString(current_piece_number) << "to original table"); - throw; - } + num_nodes = std::max(current_num_nodes, num_nodes); - try - { - String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) - { - query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + - " PARTITION " + partition_name + " DEDUPLICATE;"; + LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << partition_name + << " piece " << toString(current_piece_number) + << "to original table"); + throw; + } - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_deduplicate_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY); + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + partition_name + " DEDUPLICATE;"; - LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); - } - } - catch(...) - { - LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name << "in the original table"); - throw; + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " + << toString(num_nodes)); } } + catch (...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name + << "in the original table"); + throw; + } } + + /// Create node to signal that we finished moving { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); @@ -1011,15 +1023,28 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Try to move only if all pieces were copied. if (partition_copying_is_done) { - try + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) { - auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); - if (res == PartitionTaskStatus::Finished) - partition_moving_is_done = true; - } - catch (...) - { - tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); + try + { + auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); + if (res == PartitionTaskStatus::Finished) + { + partition_moving_is_done = true; + break; + } + + /// Sleep if this task is active + if (res == PartitionTaskStatus::Active) + std::this_thread::sleep_for(default_sleep_time); + + /// Repeat on errors + } + catch (...) { + tryLogCurrentException(log, + "Some error occured while moving pieces to destination table for partition " + + partition_name); + } } } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index ce9ea7cc7ef..8558a2859e3 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -56,6 +56,11 @@ public: copy_fault_probability = copy_fault_probability_; } + void setMoveFaultProbability(double move_fault_probability_) + { + move_fault_probability = move_fault_probability_; + } + protected: String getWorkersPath() const @@ -120,7 +125,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 5; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); @@ -201,6 +206,7 @@ private: bool is_safe_mode = false; double copy_fault_probability = 0.0; + double move_fault_probability = 0.0; Context & context; Poco::Logger * log; diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index fe5b7f72588..37b66b0fb73 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -17,6 +17,8 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) is_safe_mode = config().has("safe-mode"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); + if (config().has("move-fault-probability")) + move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0); base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); // process_id is '#_' time_t timestamp = Poco::Timestamp().epochTime(); @@ -67,6 +69,8 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) .binding("safe-mode")); options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") .argument("copy-fault-probability").binding("copy-fault-probability")); + options.addOption(Poco::Util::Option("move-fault-probability", "", "the moving fails with specified probability (used to test partition state recovering)") + .argument("move-fault-probability").binding("move-fault-probability")); options.addOption(Poco::Util::Option("log-level", "", "sets log level") .argument("log-level").binding("log-level")); options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") @@ -115,6 +119,7 @@ void ClusterCopierApp::mainImpl() auto copier = std::make_unique(task_path, host_id, default_database, *context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); + copier->setMoveFaultProbability(move_fault_probability); auto task_file = config().getString("task-file", ""); if (!task_file.empty()) diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h index 25350aefd3a..8cadd9d5dff 100644 --- a/dbms/programs/copier/ClusterCopierApp.h +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -78,7 +78,8 @@ private: std::string task_path; std::string log_level = "trace"; bool is_safe_mode = false; - double copy_fault_probability = 0; + double copy_fault_probability = 0.0; + double move_fault_probability = 0.0; bool is_help = false; std::string base_dir; diff --git a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index f00cf1cf351..54a8822fa98 100644 --- a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -57,5 +57,24 @@ + + + + s0_0_0 + 9000 + + + + + + + + + s1_0_0 + 9000 + + + + diff --git a/dbms/tests/integration/test_cluster_copier/task_trivial.xml b/dbms/tests/integration/test_cluster_copier/task_trivial.xml new file mode 100644 index 00000000000..c23b9322470 --- /dev/null +++ b/dbms/tests/integration/test_cluster_copier/task_trivial.xml @@ -0,0 +1,64 @@ + + + + 3 + + + + 1 + + + + + 0 + + + + + + + + + + source_trivial_cluster + default + trivial + + destination_trivial_cluster + default + trivial + + + ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 5 ORDER BY d SETTINGS index_granularity = 16 + + + d + 1 + + + d - d = 0 + + + + + + + + + s0_0_0 + 9000 + + + + + + + + + s1_0_0 + 9000 + + + + + + \ No newline at end of file diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 47df976bbcf..d411eba3974 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -15,6 +15,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.33 cluster = None @@ -259,17 +260,23 @@ def execute_task(task, cmd_options): # Tests -def test_copy1_simple(started_cluster): +def test_copy_simple(started_cluster): execute_task(Task1(started_cluster), []) -def test_copy1_with_recovering(started_cluster): +def test_copy_with_recovering(started_cluster): execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) +def test_copy_with_recovering_after_move_faults(started_cluster): + execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + def test_copy_month_to_week_partition(started_cluster): execute_task(Task2(started_cluster), []) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster), ['--copy-fault-probability', str(0.3)]) + execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + +def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): + execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): execute_task(Task_test_block_size(started_cluster), []) diff --git a/dbms/tests/integration/test_cluster_copier/trivial_test.py b/dbms/tests/integration/test_cluster_copier/trivial_test.py new file mode 100644 index 00000000000..aa204825462 --- /dev/null +++ b/dbms/tests/integration/test_cluster_copier/trivial_test.py @@ -0,0 +1,144 @@ +import os +import os.path as p +import sys +import time +import datetime +import pytest +from contextlib import contextmanager +import docker +from kazoo.client import KazooClient + + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.1 +cluster = None + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + clusters_schema = { + "0" : {"0" : ["0"]}, + "1" : {"0" : ["0"]} + } + + cluster = ClickHouseCluster(__file__) + + for cluster_name, shards in clusters_schema.iteritems(): + for shard_name, replicas in shards.iteritems(): + for replica_name in replicas: + name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) + cluster.add_instance(name, + config_dir="configs", + macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, + with_zookeeper=True) + + cluster.start() + yield cluster + + finally: + pass + cluster.shutdown() + + +class TaskTrivial: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path="/clickhouse-copier/task_trivial" + self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_trivial.xml'), 'r').read() + + + def start(self): + source = cluster.instances['s0_0_0'] + destination = cluster.instances['s1_0_0'] + + for node in [source, destination]: + node.query("DROP DATABASE IF EXISTS default") + node.query("CREATE DATABASE IF NOT EXISTS default") + + source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " + "PARTITION BY d % 5 ORDER BY d SETTINGS index_granularity = 16") + + source.query("INSERT INTO trivial SELECT * FROM system.numbers LIMIT 1002", settings={"insert_distributed_sync": 1}) + + + def check(self): + source = cluster.instances['s0_0_0'] + destination = cluster.instances['s1_0_0'] + + assert TSV(source.query("SELECT count() FROM trivial")) == TSV("1002\n") + assert TSV(destination.query("SELECT count() FROM trivial")) == TSV("1002\n") + + for node in [source, destination]: + node.query("DROP TABLE trivial") + + +def execute_task(task, cmd_options): + task.start() + + zk = cluster.get_kazoo_client('zoo1') + print "Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]) + + zk_task_path = task.zk_task_path + zk.ensure_path(zk_task_path) + zk.create(zk_task_path + "/description", task.copier_task_config) + + # Run cluster-copier processes on each node + docker_api = docker.from_env().api + copiers_exec_ids = [] + + cmd = ['/usr/bin/clickhouse', 'copier', + '--config', '/etc/clickhouse-server/config-copier.xml', + '--task-path', zk_task_path, + '--base-dir', '/var/log/clickhouse-server/copier'] + cmd += cmd_options + + print(cmd) + + for instance_name, instance in cluster.instances.iteritems(): + container = instance.get_docker_handle() + exec_id = docker_api.exec_create(container.id, cmd, stderr=True) + docker_api.exec_start(exec_id, detach=True) + + copiers_exec_ids.append(exec_id) + print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) + + # Wait for copiers stopping and check their return codes + for exec_id, instance in zip(copiers_exec_ids, cluster.instances.itervalues()): + while True: + res = docker_api.exec_inspect(exec_id) + if not res['Running']: + break + time.sleep(1) + + assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) + + try: + task.check() + finally: + zk.delete(zk_task_path, recursive=True) + + +# Tests + +def test_trivial_copy(started_cluster): + execute_task(TaskTrivial(started_cluster), []) + +def test_trivial_copy_with_copy_fault(started_cluster): + execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(0.1)]) + +def test_trivial_copy_with_move_fault(started_cluster): + execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(0.1)]) + + +if __name__ == '__main__': + with contextmanager(started_cluster)() as cluster: + for name, instance in cluster.instances.items(): + print name, instance.ip_address + raw_input("Cluster created, press any key to destroy...") \ No newline at end of file From 784ac0059f97c5851b09aebe91380d2cdd89e09b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 19:23:47 +0300 Subject: [PATCH 048/147] better --- dbms/programs/copier/ClusterCopier.cpp | 6 +++++- dbms/programs/copier/ClusterCopier.h | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index b1c284122a4..7edd8953252 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -596,6 +596,9 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active << " has not been successfully finished by " << status.owner << ". Will try to move by myself."); + + /// Remove is_done marker. + zookeeper->remove(current_partition_attach_is_done); } } @@ -1036,9 +1039,10 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Sleep if this task is active if (res == PartitionTaskStatus::Active) - std::this_thread::sleep_for(default_sleep_time); + break; /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); } catch (...) { tryLogCurrentException(log, diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 8558a2859e3..0f994e520d2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -125,7 +125,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 100; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7ea1243ff49..fb2ccfcd323 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4941,10 +4941,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. - /// In case of REPLACE PARTITION we can replace existing partition with empty. - if (!replace && src_parts.empty()) - return; +// /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. +// /// In case of REPLACE PARTITION we can replace existing partition with empty. +// if (!replace && src_parts.empty()) +// return; ReplicatedMergeTreeLogEntryData entry; { From 6a991783f47dd3716a8fc0716d84aaee472dd960 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 19:50:22 +0300 Subject: [PATCH 049/147] simplify code --- dbms/programs/copier/Aliases.h | 1 - dbms/programs/copier/ClusterCopier.cpp | 21 +-------------------- 2 files changed, 1 insertion(+), 21 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index d91685445f8..c4d9c40d9f1 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -2,7 +2,6 @@ #include - namespace DB { using ConfigurationPtr = Poco::AutoPtr; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 7edd8953252..a25e3b56348 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -45,13 +45,6 @@ void ClusterCopier::init() for (auto & task_table : task_cluster->table_tasks) { task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - auto pull_shards_info = task_table.cluster_pull->getShardsInfo(); - for (auto & shard_info : pull_shards_info) - { - std::cout << "current_shard " << toString(shard_info.shard_num) << "has remote connections " - << toString(shard_info.hasRemoteConnections()) << std::endl; - } - std::cout << "CLUSTER PULL " << std::endl; task_table.cluster_push = context.getCluster(task_table.cluster_push_name); task_table.initShards(task_cluster->random_engine); } @@ -417,19 +410,7 @@ bool ClusterCopier::checkPartitionPieceIsClean( if (zookeeper->exists(task_status_path, &stat)) task_start_clock = LogicalClock(stat.mzxid); - /// If statement for readability. - - LOG_INFO(log, "clean_state_clock.is_stale() " << clean_state_clock.is_stale()); - LOG_INFO(log, "clean_state_clock.is_clean() " << clean_state_clock.is_clean()); - if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - return true; - } - else - { - return false; - } - + return clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock); } From 1798bbedec264549789b5c1adf227f7f9535b346 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 21:07:54 +0300 Subject: [PATCH 050/147] style issues --- dbms/programs/copier/ClusterCopier.cpp | 40 ++++--------- dbms/programs/copier/ClusterCopier.h | 12 ++-- dbms/programs/copier/ShardPartition.h | 58 +++++++++++-------- dbms/programs/copier/ShardPartitionPiece.h | 22 +++---- dbms/programs/copier/TaskTableAndShard.h | 22 ++----- dbms/src/Interpreters/Cluster.cpp | 4 -- .../Storages/StorageReplicatedMergeTree.cpp | 6 +- 7 files changed, 71 insertions(+), 93 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index a25e3b56348..b36246e95db 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -593,8 +593,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - /// TODO: Execute alter table move partition. - LOG_DEBUG(log, "Trying to move partition " << partition_name << " piece " << toString(current_piece_number) << " to original table"); @@ -646,7 +644,8 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task try { String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) { + if (!task_table.isReplicatedTable()) + { query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + " PARTITION " + partition_name + " DEDUPLICATE;"; @@ -671,9 +670,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task } } - - - /// Create node to signal that we finished moving { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); @@ -840,19 +836,14 @@ bool ClusterCopier::tryDropPartitionPiece( LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); /// We have to drop partition_piece on each replica - UInt64 num_shards = executeQueryOnCluster( + size_t num_shards = executeQueryOnCluster( cluster_push, query, nullptr, &settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); - UNUSED(num_shards); -// if (num_shards < cluster_push->getShardCount()) -// { -// LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); -// return false; -// } + LOG_INFO(log, "DROP PARTITION was successfully executed on " << num_shards << " nodes of a cluster."); /// Update the locking node if (!my_clock.is_stale()) @@ -1025,10 +1016,9 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - catch (...) { - tryLogCurrentException(log, - "Some error occured while moving pieces to destination table for partition " + - partition_name); + catch (...) + { + tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); } } } @@ -1201,8 +1191,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + - " = " + toString(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + " = " + toString(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1238,7 +1227,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( try { - /// TODO: tryDropPartitionPiece. tryDropPartitionPiece(task_partition, current_piece_number, zookeeper, clean_state_clock); } catch (...) @@ -1317,7 +1305,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - ///TODO: Why table_split_shard??? ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; @@ -1552,8 +1539,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( zookeeper->set(current_task_piece_status_path, state_finished, 0); } - /// TODO: LOG_INFO (Piece copied and moved to destination table) - return PartitionTaskStatus::Finished; } @@ -1757,8 +1742,6 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; } -/// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); -/// Just copypaste the function above bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) { @@ -1782,7 +1765,8 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi query += " LIMIT 1"; LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + << partition_quoted_name << " piece " << std::to_string(current_piece_number) + << "existence, executing query: " << query); ParserQuery parser_query(query.data() + query.size()); ASTPtr query_ast = parseQuery(parser_query, query, 0); @@ -1792,10 +1776,10 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); if (result != 0) LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); else LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); return result != 0; } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 0f994e520d2..84b920992a5 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -16,14 +16,14 @@ class ClusterCopier { public: - ClusterCopier(String task_path_, - String host_id_, - String proxy_database_name_, + ClusterCopier(const String & task_path_, + const String & host_id_, + const String & proxy_database_name_, Context & context_) : - task_zookeeper_path(std::move(task_path_)), - host_id(std::move(host_id_)), - working_database_name(std::move(proxy_database_name_)), + task_zookeeper_path(task_path_), + host_id(host_id_), + working_database_name(proxy_database_name_), context(context_), log(&Poco::Logger::get("ClusterCopier")) {} diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index f7384b977e3..7de381977f9 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -3,36 +3,37 @@ #include "Aliases.h" #include "TaskTableAndShard.h" -namespace DB { - +namespace DB +{ /// Just destination partition of a shard /// I don't know what this comment means. /// In short, when we discovered what shards contain currently processing partition, /// This class describes a partition (name) that is stored on the shard (parent). -struct ShardPartition { +struct ShardPartition +{ ShardPartition(TaskShard &parent, String name_quoted_, size_t number_of_splits = 10) : task_shard(parent), name(std::move(name_quoted_)) { pieces.reserve(number_of_splits); } - /*useful*/ String getPartitionPath() const; + String getPartitionPath() const; - [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + String getPartitionPiecePath(size_t current_piece_number) const; - /*useful*/ String getPartitionCleanStartPath() const; + String getPartitionCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + String getPartitionPieceCleanStartPath(size_t current_piece_number) const; - /*useful*/ String getCommonPartitionIsDirtyPath() const; + String getCommonPartitionIsDirtyPath() const; - /*useful*/ String getCommonPartitionIsCleanedPath() const; + String getCommonPartitionIsCleanedPath() const; - /*??????*/ String getPartitionActiveWorkersPath() const; + String getPartitionActiveWorkersPath() const; - /*??????*/ String getActiveWorkerPath() const; + String getActiveWorkerPath() const; - /*useful*/ String getPartitionShardsPath() const; + String getPartitionShardsPath() const; - /*useful*/ String getShardStatusPath() const; + String getShardStatusPath() const; /// What partition pieces are present in current shard. /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) @@ -44,49 +45,58 @@ struct ShardPartition { String name; }; -inline String ShardPartition::getPartitionCleanStartPath() const { +inline String ShardPartition::getPartitionCleanStartPath() const +{ return getPartitionPath() + "/clean_start"; } -inline String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const { +inline String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const +{ assert(current_piece_number < task_shard.task_table.number_of_splits); return getPartitionPiecePath(current_piece_number) + "/clean_start"; } -inline String ShardPartition::getPartitionPath() const { +inline String ShardPartition::getPartitionPath() const +{ return task_shard.task_table.getPartitionPath(name); } -inline String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const { +inline String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const +{ assert(current_piece_number < task_shard.task_table.number_of_splits); return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); } -inline String ShardPartition::getShardStatusPath() const { +inline String ShardPartition::getShardStatusPath() const +{ // schema: //tables/
//shards/ // e.g. /root/table_test.hits/201701/shards/1 return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); } -inline String ShardPartition::getPartitionShardsPath() const { +inline String ShardPartition::getPartitionShardsPath() const +{ return getPartitionPath() + "/shards"; } -inline String ShardPartition::getPartitionActiveWorkersPath() const { +inline String ShardPartition::getPartitionActiveWorkersPath() const +{ return getPartitionPath() + "/partition_active_workers"; } -inline String ShardPartition::getActiveWorkerPath() const { +inline String ShardPartition::getActiveWorkerPath() const +{ return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); } -inline String ShardPartition::getCommonPartitionIsDirtyPath() const { +inline String ShardPartition::getCommonPartitionIsDirtyPath() const +{ return getPartitionPath() + "/is_dirty"; } -inline String ShardPartition::getCommonPartitionIsCleanedPath() const { +inline String ShardPartition::getCommonPartitionIsCleanedPath() const +{ return getCommonPartitionIsDirtyPath() + "/cleaned"; } - } diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 88f1ceddb0d..a21fd531da4 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -2,30 +2,32 @@ #include "Internals.h" -namespace DB { +namespace DB +{ -struct ShardPartitionPiece { +struct ShardPartitionPiece +{ ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_present_piece_) : is_absent_piece(!is_present_piece_), current_piece_number(current_piece_number_), shard_partition(parent) {} - [[maybe_unused]] String getPartitionPiecePath() const; + String getPartitionPiecePath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath() const; + String getPartitionPieceCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceIsDirtyPath() const; + String getPartitionPieceIsDirtyPath() const; - [[maybe_unused]] String getPartitionPieceIsCleanedPath() const; + String getPartitionPieceIsCleanedPath() const; - [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const; + String getPartitionPieceActiveWorkersPath() const; - [[maybe_unused]] String getActiveWorkerPath() const ; + String getActiveWorkerPath() const ; /// On what shards do we have current partition. - [[maybe_unused]] String getPartitionPieceShardsPath() const; + String getPartitionPieceShardsPath() const; - [[maybe_unused]] String getShardStatusPath() const; + String getShardStatusPath() const; String getPartitionPieceCleanerPath() const; diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 9a09deb911f..8e67cc5053b 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -14,7 +14,8 @@ namespace ErrorCodes struct TaskShard; -struct TaskTable { +struct TaskTable + { TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); @@ -43,8 +44,6 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; - bool isReplicatedTable() const { return engine_push_zk_path != ""; } @@ -134,7 +133,8 @@ struct TaskTable { /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String &partition_name) { + ClusterPartition & getClusterPartition(const String &partition_name) + { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) throw Exception("There are no cluster partition " + partition_name + " in " + table_id, @@ -146,7 +146,7 @@ struct TaskTable { UInt64 bytes_copied = 0; UInt64 rows_copied = 0; - template + template void initShards(RandomEngine &&random_engine); }; @@ -188,9 +188,7 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; - DatabaseAndTableName main_table_split_shard; - ListOfDatabasesAndTableNames list_of_split_tables_on_shard; }; @@ -228,7 +226,7 @@ inline String TaskTable::getCertainPartitionPieceIsDirtyPath(const String & part return getPartitionPiecePath(partition_name, piece_number) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const +inline String TaskTable::getCertainPartitionIsCleanedPath(const String & partition_name) const { return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } @@ -374,12 +372,6 @@ inline void TaskTable::initShards(RandomEngine && random_engine) local_shards.assign(all_shards.begin(), it_first_remote); } -inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const -{ - assert (engine_push_zk_path != ""); - return engine_push_zk_path + "/piece_" + toString(piece_number); -} - inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() { ASTPtr prev_engine_push_ast = engine_push_ast->clone(); @@ -416,6 +408,4 @@ inline String DB::TaskShard::getHostNameExample() const return replicas.at(0).readableString(); } - - } diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index dc811ffa327..f1790249cc0 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -75,8 +75,6 @@ bool Cluster::Address::isLocal(UInt16 clickhouse_port) const Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_, UInt32 replica_index_) : shard_index(shard_index_), replica_index(replica_index_) { - - std::cout << "FROM Address constructor " << " shard index " << shard_index_ << " replica index " << replica_index_ << std::endl; host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); if (config.has(config_prefix + ".user")) @@ -340,8 +338,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting if (startsWith(replica_key, "replica")) { replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num); - - std::cout << "replica num " << current_replica_num << " is a replica of shard number " << current_shard_num << std::endl; ++current_replica_num; if (!replica_addresses.back().is_local) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index fb2ccfcd323..77b306d9d97 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,6 +4861,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { + /// First argument is true, because we possibly will add new data to current table. auto lock1 = lockStructureForShare(true, context.getCurrentQueryId()); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); @@ -4941,11 +4942,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } -// /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. -// /// In case of REPLACE PARTITION we can replace existing partition with empty. -// if (!replace && src_parts.empty()) -// return; - ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From 134ac8c9b4ff79d2a299ab3a6b4f3f28fec1d84b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 11:49:05 +0300 Subject: [PATCH 051/147] tune tests --- .../integration/test_cluster_copier/task0_description.xml | 2 ++ .../test_cluster_copier/task_month_to_week_description.xml | 2 ++ dbms/tests/integration/test_cluster_copier/trivial_test.py | 4 ++-- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/tests/integration/test_cluster_copier/task0_description.xml b/dbms/tests/integration/test_cluster_copier/task0_description.xml index e8e4df99254..6fa490cd02f 100644 --- a/dbms/tests/integration/test_cluster_copier/task0_description.xml +++ b/dbms/tests/integration/test_cluster_copier/task0_description.xml @@ -28,6 +28,8 @@ default hits + 2 + 3 4 5 6 1 2 0 diff --git a/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml index e212d1a3d04..ee134603310 100644 --- a/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -30,6 +30,8 @@ --> + 2 + ENGINE= ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') diff --git a/dbms/tests/integration/test_cluster_copier/trivial_test.py b/dbms/tests/integration/test_cluster_copier/trivial_test.py index aa204825462..9b71b08674c 100644 --- a/dbms/tests/integration/test_cluster_copier/trivial_test.py +++ b/dbms/tests/integration/test_cluster_copier/trivial_test.py @@ -131,10 +131,10 @@ def test_trivial_copy(started_cluster): execute_task(TaskTrivial(started_cluster), []) def test_trivial_copy_with_copy_fault(started_cluster): - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(0.1)]) + execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) def test_trivial_copy_with_move_fault(started_cluster): - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(0.1)]) + execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) if __name__ == '__main__': From 6f526db293558f49fab920e5bba7ee5dddba93e1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 16:25:49 +0300 Subject: [PATCH 052/147] also better code --- dbms/programs/copier/ClusterCopier.cpp | 89 +++++++++++------------- dbms/programs/copier/ClusterCopier.h | 17 +++-- dbms/programs/copier/Internals.cpp | 19 +++-- dbms/programs/copier/Internals.h | 12 +--- dbms/programs/copier/TaskTableAndShard.h | 15 +--- dbms/src/Interpreters/Cluster.h | 12 ---- 6 files changed, 65 insertions(+), 99 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index b36246e95db..df524b6d314 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,8 +1,6 @@ #include "ClusterCopier.h" -#include "Internals.h" - -#include +ls#include "Internals.h" #include #include @@ -523,7 +521,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } -PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) +TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) { bool inject_fault = false; if (move_fault_probability > 0) @@ -551,7 +549,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task if (e.code == Coordination::ZNODEEXISTS) { LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active); - return PartitionTaskStatus::Active; + return TaskStatus::Active; } throw; @@ -569,7 +567,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task { LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active << " has been successfully moved to destination table by " << status.owner); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. @@ -614,10 +612,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task try { - size_t num_nodes = 0; - - ///FIXME: We have to be sure that every node in cluster executed this query - size_t current_num_nodes = executeQueryOnCluster( + size_t num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, nullptr, @@ -625,9 +620,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); - num_nodes = std::max(current_num_nodes, num_nodes); - - LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) @@ -676,7 +668,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task zookeeper->set(current_partition_attach_is_done, state_finished, 0); } - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Removes MATERIALIZED and ALIAS columns from create table query @@ -729,7 +721,6 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage( res->set(res->columns_list, create.columns_list->clone()); res->set(res->storage, new_storage_ast->clone()); - return res; } @@ -949,7 +940,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Do not sleep if there is a sequence of already processed shards to increase startup bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; + TaskStatus task_status = TaskStatus::Error; bool was_error = false; has_shard_to_process = true; for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) @@ -957,20 +948,20 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); /// Exit if success - if (task_status == PartitionTaskStatus::Finished) + if (task_status == TaskStatus::Finished) break; was_error = true; /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) + if (task_status == TaskStatus::Active) break; /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - if (task_status == PartitionTaskStatus::Error) + if (task_status == TaskStatus::Error) ++num_failed_shards; previous_shard_is_instantly_finished = !was_error; @@ -1003,17 +994,18 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab try { auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); - if (res == PartitionTaskStatus::Finished) + /// Exit and mark current task is done. + if (res == TaskStatus::Finished) { partition_moving_is_done = true; break; } - /// Sleep if this task is active - if (res == PartitionTaskStatus::Active) + /// Exit if this task is active. + if (res == TaskStatus::Active) break; - /// Repeat on errors + /// Repeat on errors. std::this_thread::sleep_for(default_sleep_time); } catch (...) @@ -1064,9 +1056,9 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab } /// Job for copying partition from particular shard. -PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) +TaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res; + TaskStatus res; try { @@ -1075,7 +1067,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo catch (...) { tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; + res = TaskStatus::Error; } /// At the end of each task check if the config is updated @@ -1091,12 +1083,12 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo return res; } -PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, +TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - PartitionTaskStatus res{PartitionTaskStatus::Finished}; + TaskStatus res{TaskStatus::Finished}; bool was_failed_pieces = false; bool was_active_pieces = false; @@ -1111,33 +1103,32 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); /// Exit if success - if (res == PartitionTaskStatus::Finished) + if (res == TaskStatus::Finished) break; /// Skip if the task is being processed by someone - if (res == PartitionTaskStatus::Active) + if (res == TaskStatus::Active) break; /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - was_active_pieces = (res == PartitionTaskStatus::Active); - was_failed_pieces = (res == PartitionTaskStatus::Error); + was_active_pieces = (res == TaskStatus::Active); + was_failed_pieces = (res == TaskStatus::Error); } if (was_failed_pieces) - return PartitionTaskStatus::Error; + return TaskStatus::Error; if (was_active_pieces) - return PartitionTaskStatus::Active; + return TaskStatus::Active; - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } -/*...*/ -PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( +TaskStatus ClusterCopier::processPartitionPieceTaskImpl( const ConnectionTimeouts & timeouts, ShardPartition & task_partition, const size_t current_piece_number, bool is_unprioritized_task) { @@ -1234,7 +1225,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( tryLogCurrentException(log, "An error occurred when clean partition"); } - return PartitionTaskStatus::Error; + return TaskStatus::Error; } /// Create ephemeral node to mark that we are active and process the partition @@ -1249,7 +1240,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (e.code == Coordination::ZNODEEXISTS) { LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); - return PartitionTaskStatus::Active; + return TaskStatus::Active; } throw; @@ -1266,7 +1257,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. @@ -1276,7 +1267,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } @@ -1293,7 +1284,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (res == Coordination::ZOK) LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Check that destination partition is empty if we are first worker @@ -1334,7 +1325,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } zookeeper->set(partition_piece.getPartitionPieceCleanStartPath(), "ok"); @@ -1350,14 +1341,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); } @@ -1491,7 +1482,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } @@ -1527,19 +1518,19 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " became dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } zookeeper->set(current_task_piece_status_path, state_finished, 0); } - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 84b920992a5..60d63e37cf6 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -111,7 +111,7 @@ protected: /*Alter successful insertion to helping tables it will move all pieces to destination table*/ - PartitionTaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); + TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); @@ -130,13 +130,18 @@ protected: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); + TaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + bool is_unprioritized_task); - PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - bool is_unprioritized_task); + TaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + bool is_unprioritized_task); - PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task); + TaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + const size_t current_piece_number, + bool is_unprioritized_task); void dropAndCreateLocalTable(const ASTPtr & create_ast); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 263bf5f8590..af41ce3ebbe 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -168,16 +168,15 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast) } -String createCommaSeparatedStringFrom(const Names & strings) +String createCommaSeparatedStringFrom(const Names & names) { - String answer; - for (auto & string: strings) - answer += string + ", "; - - /// Remove last comma and space - answer.pop_back(); - answer.pop_back(); - return answer; + std::ostringstream ss; + if(!names.empty()) + { + std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); + ss << names.back(); + } + return ss.str(); } Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) @@ -189,7 +188,7 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) const auto primary_key_expr_list = primary_key_ast ? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone(); - /// VersionedCollapsing ??? + /// Maybe we have to handle VersionedCollapsing engine separately. But in our case in looks pointless. size_t primary_key_size = primary_key_expr_list->children.size(); size_t sorting_key_size = sorting_key_expr_list->children.size(); diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index fc4c2f09da7..63ef9483228 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -145,15 +145,9 @@ struct ShardPriority } }; -/// Execution status of a task -enum class PartitionTaskStatus -{ - Active, - Finished, - Error, -}; - -enum class PartititonPieceTaskStatus +/// Execution status of a task. +/// Is used for: partition copying task status, partition piece copying task status, partition moving task status. +enum class TaskStatus { Active, Finished, diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 8e67cc5053b..615ad297b79 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -15,7 +15,7 @@ namespace ErrorCodes struct TaskShard; struct TaskTable - { +{ TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); @@ -87,17 +87,6 @@ struct TaskTable ASTPtr sharding_key_ast; ASTPtr main_engine_split_ast; - - /* - * Auxuliary table engines used to perform partition piece copying. - * Each AST represent table engine for certatin piece number. - * After copying partition piece is Ok, this piece will be moved to the main - * target table. All this tables are stored on each shard as the main table. - * We have to use separate tables for partition pieces because of the atomicity of copying. - * Also if we want to move some partition to another table, the partition keys have to be the same. - * */ - - /* * To copy partiton piece form one cluster to another we have to use Distributed table. * In case of usage separate table (engine_push) for each partiton piece, @@ -133,7 +122,7 @@ struct TaskTable /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String &partition_name) + ClusterPartition & getClusterPartition(const String & partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 35fe088a216..b029580969f 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -162,18 +162,6 @@ public: /// The number of all shards. size_t getShardCount() const { return shards_info.size(); } -// /// The number of all nodes (all replicas of each shard). -// size_t getNodesCount() const -// { -// size_t nodes_count = 0; -// std::cout << "addresses_with_failover.size() " << addresses_with_failover.size() << std::endl; -// std::for_each(addresses_with_failover.begin(), addresses_with_failover.end(), [&] (const Addresses & addresses) -// { -// nodes_count += addresses.size(); -// }); -// return nodes_count; -// } - /// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster. std::unique_ptr getClusterWithSingleShard(size_t index) const; From 23f027e6dadbf60bd4d1d8d6dc2142ddfc64fbb0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 17:40:24 +0300 Subject: [PATCH 053/147] typo --- dbms/programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index df524b6d314..d3b9d0c3512 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,6 +1,6 @@ #include "ClusterCopier.h" -ls#include "Internals.h" +#include "Internals.h" #include #include From c33771105fefc17a1da7116cca03abfe690f826a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 21:35:58 +0300 Subject: [PATCH 054/147] partition deletion added --- dbms/programs/copier/ClusterCopier.cpp | 35 ++++++++++++++++++++------ dbms/programs/copier/ClusterCopier.h | 5 ++++ 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d3b9d0c3512..915cf36fa7c 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1580,6 +1580,34 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) } } + +void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name) +{ + LOG_DEBUG(log, "Try drop partition partition from all helping tables."); + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "ALTER TABLE " + getQuotedTable(helping_table) + " DROP PARTITION " + partition_name; + + const ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// We have to drop partition_piece on each replica + UInt64 num_nodes = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + LOG_DEBUG(log, "DROP PARTITION query was successfully executed on " << toString(num_nodes) << " nodes."); + } + LOG_DEBUG(log, "All helping tables dropped partition " << partition_name); +} + String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); @@ -1655,8 +1683,6 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout task_shard.list_of_split_tables_on_shard[piece_number], storage_piece_split_ast); - std::cout << "anime" << queryToString(create_table_split_piece_ast) << std::endl; - dropAndCreateLocalTable(create_table_split_piece_ast); } } @@ -1819,11 +1845,6 @@ UInt64 ClusterCopier::executeQueryOnCluster( UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - for (size_t i = 0; i < num_replicas; ++i) - { - std::cout << "host_name " << cluster->getShardsAddresses().at(shard_index)[i].host_name - << " port " << cluster->getShardsAddresses().at(shard_index)[i].port << std::endl; - } origin_replicas_number += num_replicas; UInt64 num_local_replicas = shard.getLocalNodeCount(); UInt64 num_remote_replicas = num_replicas - num_local_replicas; diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 60d63e37cf6..53415df515c 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -149,6 +149,11 @@ protected: void dropHelpingTables(const TaskTable & task_table); + /// Is used for usage less disk space. + /// After all pieces were successfully moved to original destination + /// table we can get rid of partition pieces (partitions in helping tables). + void dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); From fdd12bb8d7a9d9882ff0924a460750dd58d41642 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 22:19:37 +0300 Subject: [PATCH 055/147] remote TODO --- dbms/programs/copier/ClusterCopier.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 228e33b7b62..12ac5bdf47e 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -155,15 +155,12 @@ protected: ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - /// TODO: rewrite comment void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name); - /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); - /// Just copypaste the function above bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); From 0a7f4e08cf803577cc2e42fde5f03d825dabd301 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 23:18:01 +0300 Subject: [PATCH 056/147] style check fix --- dbms/programs/copier/Internals.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index af41ce3ebbe..545df2e779c 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -171,7 +171,7 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast) String createCommaSeparatedStringFrom(const Names & names) { std::ostringstream ss; - if(!names.empty()) + if (!names.empty()) { std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); ss << names.back(); From b623e3425b3e02453dd1f6669553adc8b8a08c9d Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 19 Mar 2020 02:01:43 +0400 Subject: [PATCH 057/147] fix build --- dbms/programs/copier/Internals.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 64a4fafd490..8e4f7afeb6e 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -198,7 +198,7 @@ ASTPtr extractPrimaryKey(const ASTPtr & storage_ast); ASTPtr extractOrderBy(const ASTPtr & storage_ast); -String createCommaSeparatedStringFrom(const Names & strings); +String createCommaSeparatedStringFrom(const Names & names); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); From f08771e563e4d59f9f43ad418c3cc67460060964 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 19 Mar 2020 04:15:01 +0300 Subject: [PATCH 058/147] parse tuple as literal if possible --- dbms/src/Parsers/ASTLiteral.cpp | 12 +++-- dbms/src/Parsers/ExpressionElementParsers.cpp | 26 +++++++---- dbms/src/Parsers/ExpressionElementParsers.h | 45 ++++++++++++++++--- dbms/src/Parsers/ParserPartition.cpp | 27 ++++++++--- dbms/tests/performance/set_index.xml | 13 +++++- 5 files changed, 97 insertions(+), 26 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 04504fb3594..1a75ad019e5 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -16,17 +16,21 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { - /// Special case for very large arrays. Instead of listing all elements, will use hash of them. + /// 100 - just arbitrary value. + constexpr auto MIN_ELEMENTS_FOR_HASHING = 100; + + /// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) - if (value.getType() == Field::Types::Array - && value.get().size() > 100) /// 100 - just arbitrary value. + auto type = value.getType(); + if ((type == Field::Types::Array && value.get().size() > MIN_ELEMENTS_FOR_HASHING) + || (type == Field::Types::Tuple && value.get().size() > MIN_ELEMENTS_FOR_HASHING)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); UInt64 low, high; hash.get128(low, high); - writeCString("__array_", ostr); + writeCString(type == Field::Types::Array ? "__array_" : "__tuple_", ostr); writeText(low, ostr); ostr.write('_'); writeText(high, ostr); diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index eb77d77a5c8..af676d94f77 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -990,15 +990,15 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte return true; } - -bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +template +bool ParserCollectionOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (pos->type != TokenType::OpeningSquareBracket) + if (pos->type != opening_bracket) return false; Pos literal_begin = pos; - Array arr; + Collection arr; ParserLiteral literal_p; @@ -1008,9 +1008,16 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { if (!arr.empty()) { - if (pos->type == TokenType::ClosingSquareBracket) + if (pos->type == closing_bracket) { - auto literal = std::make_shared(arr); + std::shared_ptr literal; + + /// Parse one-element tuples (e.g. (1)) as single values for backward compatibility. + if (std::is_same_v && arr.size() == 1) + literal = std::make_shared(arr[0]); + else + literal = std::make_shared(arr); + literal->begin = literal_begin; literal->end = ++pos; node = literal; @@ -1022,7 +1029,9 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } else { - expected.add(pos, "comma or closing square bracket"); + std::stringstream msg; + msg << "comma or " << getTokenName(closing_bracket); + expected.add(pos, msg.str().c_str()); return false; } } @@ -1034,7 +1043,7 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec arr.push_back(literal_node->as().value); } - expected.add(pos, "closing square bracket"); + expected.add(pos, getTokenName(closing_bracket)); return false; } @@ -1235,6 +1244,7 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserSubquery().parse(pos, node, expected) + || ParserTupleOfLiterals().parse(pos, node, expected) || ParserParenthesisExpression().parse(pos, node, expected) || ParserArrayOfLiterals().parse(pos, node, expected) || ParserArray().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index c82dfd7a62f..b2923118225 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -217,17 +218,49 @@ protected: }; -/** An array of literals. - * Arrays can also be parsed as an application of [] operator. - * But parsing the whole array as a whole constant seriously speeds up the analysis of expressions in the case of very large arrays. - * We try to parse the array as an array of literals first (fast path), - * and if it did not work out (when the array consists of complex expressions) - parse as an application of [] operator (slow path). +/** An array or tuple of literals. + * Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function. + * But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection. + * We try to parse the array or tuple as a collection of literals first (fast path), + * and if it did not work out (when the collection consists of complex expressions) - + * parse as an application of [] operator or 'tuple' function (slow path). */ +template +class ParserCollectionOfLiterals : public IParserBase +{ +public: + ParserCollectionOfLiterals(TokenType opening_bracket_, TokenType closing_bracket_) + : opening_bracket(opening_bracket_), closing_bracket(closing_bracket_) {} +protected: + const char * getName() const override { return "collection of literals"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +private: + TokenType opening_bracket; + TokenType closing_bracket; +}; + +class ParserTupleOfLiterals : public IParserBase +{ +public: + ParserCollectionOfLiterals tuple_parser{TokenType::OpeningRoundBracket, TokenType::ClosingRoundBracket}; +protected: + const char * getName() const override { return "tuple"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return tuple_parser.parse(pos, node, expected); + } +}; + class ParserArrayOfLiterals : public IParserBase { +public: + ParserCollectionOfLiterals array_parser{TokenType::OpeningSquareBracket, TokenType::ClosingSquareBracket}; protected: const char * getName() const override { return "array"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return array_parser.parse(pos, node, expected); + } }; diff --git a/dbms/src/Parsers/ParserPartition.cpp b/dbms/src/Parsers/ParserPartition.cpp index 511db9a53b3..d5ba5739ab1 100644 --- a/dbms/src/Parsers/ParserPartition.cpp +++ b/dbms/src/Parsers/ParserPartition.cpp @@ -38,14 +38,34 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) String fields_str; const auto * tuple_ast = value->as(); + bool surrounded_by_parens = false; if (tuple_ast && tuple_ast->name == "tuple") { + surrounded_by_parens = true; const auto * arguments_ast = tuple_ast->arguments->as(); if (arguments_ast) fields_count = arguments_ast->children.size(); else fields_count = 0; + } + else if (auto literal = value->as()) + { + if (literal->value.getType() == Field::Types::Tuple) + { + surrounded_by_parens = true; + fields_count = literal->value.get().size(); + } + else + { + fields_count = 1; + fields_str = String(begin->begin, pos->begin - begin->begin); + } + } + else + return false; + if (surrounded_by_parens) + { Pos left_paren = begin; Pos right_paren = pos; @@ -61,13 +81,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) fields_str = String(left_paren->end, right_paren->begin - left_paren->end); } - else if (value->as()) - { - fields_count = 1; - fields_str = String(begin->begin, pos->begin - begin->begin); - } - else - return false; partition->value = value; partition->children.push_back(value); diff --git a/dbms/tests/performance/set_index.xml b/dbms/tests/performance/set_index.xml index e5848b08124..090d8ac8c08 100644 --- a/dbms/tests/performance/set_index.xml +++ b/dbms/tests/performance/set_index.xml @@ -13,8 +13,19 @@ - + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM zeros(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + + SELECT count() FROM numbers(1000) WHERE toString(number) IN ('41577', '83972', '51697', '50014', '37553', '93459', '87438', '95971', '83186', '74326', '67871', '50406', '83678', '29655', '18580', '83905', '61518', '29059', '56700', '82787', '98672', '30884', '81822', '39850', '80852', '57627', '91346', '64522', '17781', '49467', '41099', '41929', '85618', '91389', '68564', '91769', '81219', '52218', '37220', '97097', '2129', '9886', '52049', '34847', '25364', '36429', '76897', '71868', '58121', '71199', '84819', '69991', '34046', '64507', '34892', '24228', '36986', '28588', '51159', '53444', '80531', '9941', '20256', '48103', '32565', '62890', '5379', '60302', '46434', '3205', '18821', '31030', '19794', '71557', '71703', '15024', '14004', '82164', '95659', '40227', '83358', '24395', '9610', '19814', '48491', '66412', '16012', '71586', '42143', '51103', '24463', '89949', '35694', '39193', '63904', '40489', '77144', '94014', '84836', '9980', '46554', '43905', '25588', '25205', '72624', '10249', '35888', '98478', '99030', '26834', '31', '81499', '14847', '82997', '92357', '92893', '17426', '56630', '22252', '68119', '62710', '8740', '82144', '79916', '23391', '30192', '99271', '96435', '44237', '98327', '69481', '16691', '13643', '84554', '38571', '70926', '99283', '79000', '20926', '86495', '4834', '1222', '39486', '57697', '58002', '40790', '15623', '3999', '31515', '12694', '26143', '35951', '54085', '97534', '35329', '73535', '88715', '29572', '75799', '45166', '32066', '48023', '69523', '93150', '8740', '96790', '15534', '63252', '5142', '67045', '93992', '16663', '292', '63924', '6588', '12190', '31506', '69590', '35394', '55168', '65223', '79183', '32600', '69676', '28316', '72111', '53531', '15073', '41127', '73451', '24725', '61647', '65315', '41143', '26493', '95608', '34407', '76098', '53105', '83691', '48755', '35696', '62587', '81826', '3963', '45766', '82751', '12430', '97685', '29919', '78155', '71636', '50215', '89734', '9892', '47151', '54855', '3428', '9712', '52592', '2403', '79602', '81243', '79859', '57361', '82000', '42107', '28860', '99591', '28296', '57337', '64969', '32332', '25535', '30924', '21313', '32554', '17342', '87311', '19825', '24898', '61323', '83209', '79322', '79009', '50746', '33396', '62033', '16548', '17427', '24073', '34640', '52368', '4724', '80408', '40', '33787', '16666', '19665', '86751', '27264', '2241', '88134', '53566', '10589', '79711', '92823', '58972', '91767', '60885', '51659', '7867', '96849', '30360', '20914', '9584', '1250', '22871', '23282', '99312', '4683', '33429', '68361', '82614', '81440', '47863', '69790', '11968', '75210', '66854', '37002', '61142', '71514', '1588', '42336', '11069', '26291', '2261', '71056', '13492', '9133', '91216', '72207', '71586', '86535', '83898', '24392', '45384', '48545', '61972', '503', '80180', '35834', '97025', '70411', '55039', '35430', '27631', '82533', '96831', '74077', '42533', '14451', '26943', '53783', '69489', '71969', '8432', '37230', '61348', '19472', '59115', '9886', '50951', '57109', '7141', '1902', '84130', '4323', '55889', '47784', '2220', '75988', '66988', '63721', '8131', '95601', '95207', '2311', '26541', '50991', '6717', '2969', '71857', '51034', '65958', '94716', '90275', '21012', '46859', '7984', '31131', '46457', '69578', '44540', '7294', '80117', '9925', '60155', '90608', '82684', '32193', '87071', '28006', '87604', '24501', '79087', '2848', '29237', '11221', '81319', '40966', '87641', '35325', '78705', '88636', '78717', '62831', '56390', '99271', '43821', '14453', '17923', '62695', '77322', '21038', '67677', '41271', '4376', '65426', '46091', '19887', '97251', '55583', '58763', '3826', '35037', '73533', '64267', '82319', '9836', '42622', '96829', '16363', '10455', '49290', '99992', '98229', '66356', '59087', '73998', '25986', '4279', '56790', '69540', '588', '36620', '60358', '45056', '89297', '42740', '8323', '19245', '82417', '41431', '699', '11554', '73910', '44491', '56019', '68901', '45816', '68126', '89379', '23885', '13263', '56395', '73130', '19089', '23771', '10335', '48547', '16903', '6453', '33560', '89668', '38159', '43177', '90655', '49712', '62', '66920', '34180', '12150', '48564', '39538', '85026', '87195', '14928', '8956', '71157', '53287', '39161', '67583', '83309', '92054', '86977', '56188', '15229', '88170', '60894', '58497', '89254', '40082', '86890', '60161', '97291', '45878', '23368', '14577', '92870', '37017', '97356', '99426', '76061', '89186', '99751', '85153', '61580', '39360', '90107', '25603', '26798', '76224', '6469', '7912', '69838', '16404', '67497', '28965', '80836', '80365', '91249', '48713', '17113', '33090', '40793', '70450', '66689', '83698', '17802', '43869', '13355', '18959', '79411', '87930', '9265', '37504', '44876', '97234', '94149', '35040', '22049', '49248', '6535', '36080', '28346', '94437', '78319', '17961', '89056', '56161', '35810', '41632', '45494', '53351', '89729', '99510', '51584', '59688', '6193', '70809', '51093', '92589', '90247', '34910', '78235', '17362', '49423', '63324', '525', '37638', '72325', '89356', '15298', '59116', '17848', '65429', '27029', '84781', '70247', '8825', '35082', '70451', '22522', '58125', '91879', '90531', '2478', '463', '37902', '54405', '87267', '72688', '22803', '33134', '35177', '84551', '44974', '88375', '76407', '27774', '33849', '19915', '82014', '80434', '26380', '48777', '53811', '14838', '26829', '56441', '99869', '49574', '85476', '19723', '16907', '4018', '37338', '78510', '47912', '13030', '65277', '95716', '67363', '21393', '89887', '78842', '81650', '903', '17436', '30704', '49223', '27198', '25500', '52214', '54258', '70082', '53950', '49312', '43615', '99473', '94348', '53661', '96213', '96346', '62010', '38268', '32861', '75660', '10392', '89491', '68335', '29817', '88706', '24184', '36298', '43440', '21626', '26535', '44560', '46363', '12534', '99070', '95606', '33714', '73070', '8303', '29853', '23014', '99982', '4530', '14955', '45803', '50', '90750', '30394', '81276', '95563', '47314', '58520', '91299', '88944', '54402', '67405', '29253', '47079', '71734', '99728', '17652', '13307', '35556', '18962', '26780', '17771', '53712', '60055', '37628', '35830', '90739', '61151', '41309', '27652', '3051', '53167', '98417', '19382', '36833', '75085', '65374', '87732', '30352', '31776', '32765', '97565', '92199', '49050', '29503', '51024', '18834', '8515', '24069', '96216', '10777', '90680', '18974', '68884', '85305', '36007', '56707', '4212', '47352', '34426', '13185', '92939', '95782', '70577', '58080', '98279', '3906', '5065', '56896', '16382', '31273', '17117', '98602', '12786', '24086', '63970', '72756', '35798', '82367', '7356', '53398', '68503', '2962', '16425', '67334', '68461', '65439', '15620', '70906', '29649', '46461', '74602', '38012', '71714', '16825', '89480', '53386', '88532', '35104', '28556', '82120', '23155', '23347', '24797', '60061', '54962', '99427', '82248', '82447', '39968', '63727', '27431', '81511', '91168', '71425', '80740', '84127', '40717', '15503', '15419', '46594', '61263', '19212', '53175', '70724', '74445', '23034', '71818', '40246', '18886', '53066', '4880', '83701', '86107', '87862', '44751', '392', '73440', '90291', '93395', '20894', '38463', '32664', '55158', '20090', '50004', '79070', '98471', '85478', '96615', '68149', '78334', '97752', '73207', '71678', '91238', '96757', '82598', '194', '35797', '45120', '60782', '28721', '17676', '78066', '60957', '11826', '51563', '50516', '16485', '47053', '31738', '48923', '23554', '96850', '42033', '73701', '78607', '45979', '54571', '12415', '31693', '15356', '36902', '9126', '3767', '3295', '90402', '24005', '95350', '67033', '49137', '72606', '51899', '17522', '31957', '44641', '53982', '23767', '68257', '15766', '19995', '2107', '48788', '11765', '91055', '46576', '54651', '50381', '62827', '73636', '46606', '98753', '37631', '70441', '87916', '66983', '33870', '31125', '12904', '57040', '4874', '58632', '42037', '18782', '5998', '18974', '57949', '81010', '90407', '99874', '20462', '89949', '10952', '71454', '95130', '46115', '3518', '13384', '69039', '79482', '22076', '59782', '32042', '40930', '60243', '29298', '6790', '46985', '44398', '85631', '14380', '66179', '2629', '32126', '49833', '14118', '58492', '31493', '81172', '96638', '8745', '89663', '76842', '78633', '41373', '83721', '42886', '11123', '32739', '11051', '1303', '92314', '83324', '85600', '44276', '69064', '56125', '84650', '31028', '12628', '14502', '64764', '39405', '44855', '79046', '51716', '46824', '83389', '1941', '1257', '9280', '73176', '84729', '2579', '63366', '22606', '35541', '51096', '13447', '18355', '68037', '28436', '94116', '81070', '78355', '67897', '5296', '32742', '77645', '91853', '18767', '67949', '40963', '5792', '17278', '25597', '41884', '80829', '7099', '18645', '60295', '12082', '81800', '78415', '18082', '38789', '16295', '72377', '74949', '55583', '66853', '15402', '72977', '15123', '99434', '34999', '21687', '76049', '42987', '83748', '88256', '66688', '21766', '20304', '29271', '10069', '19822', '11792', '42526', '74143', '17289', '30253', '6367', '20888', '12975', '94073', '98639', '30134', '26320', '65507', '69002', '53120', '4550', '38893', '18954', '38283', '54863', '17698', '99670', '10521', '92467', '60994', '18052', '48673', '35811', '87282', '62706', '16061', '53112', '22652', '37780', '55662', '26331', '49410', '79074', '10623', '69577', '79613', '9491', '31229', '43922', '84231', '58409', '36386', '46875', '74431', '76735', '38776', '23350', '7314', '9079', '51519', '98544', '70216', '63380', '90381', '1295', '46901', '58225', '55339', '89918', '75522', '35431', '89460', '49552', '89302', '23068', '28493', '3042', '25194', '59520', '9810', '95706', '81297', '89638', '54794', '94527', '45262', '97932', '78685', '6947', '22818', '48700', '9153', '12289', '22011', '58825', '93854', '65438', '4509', '33741', '28208', '69061', '48578', '40247', '77725', '31837', '39003', '69363', '78113', '76398', '97262', '67795', + '68446', '58896', '60969', '19849', '6722', '91854', '49519', '13949', '67109', '48824', '31723', '75554', '69575', '94986', '75350', '18628', '15284', '41943', '15433', '52607', '41', '22340', '29528', '24059', '34145', '72517', '46316', '10667', '54510', '19882', '47764', '69124', '41963', '84350', '48420', '4646', '24958', '69020', '97121', '26178', '62664', '50622', '32554', '49655', '45398', '11267', '72222', '73363', '89554', '89046', '57384', '29259', '37888', '24850', '74353', '57343', '34762', '2900', '11393', '42154', '94306', '70552', '75265', '921', '26003', '64352', '89857', '83171', '58249', '48940', '53512', '66335', '44865', '68729', '19299', '58003', '39854', '99122', '3860', '80173', '52242', '90966', '53183', '71982', '82325', '87842', '15000', '55627', '71132', '6354', '42402', '91719', '91644', '94533', '74925', '66278', '66911', '85576', '40495', '70919', '71797', '87835', '29845', '71832', '3390', '7994', '33499', '70715', '54897', '82710', '63077', '78105', '24758', '89585', '84607', '46477', '78618', '10989', '39222', '98749', '51685', '94664', '31008', '32823', '89521', '72160', '26952', '4001', '21421', '5737', '74027', '88179', '45553', '83743', '19913', '49435', '65616', '82641', '5149', '76959', '40681', '73023', '2670', '30845', '18863', '35094', '88400', '80963', '9154', '16571', '64192', '59694', '41317', '59942', '58856', '99281', '67260', '66971', '22716', '76089', '58047', '67071', '53707', '462', '52518', '72277', '10681', '69', '98855', '12593', '88842', '67242', '73543', '37439', '18413', '67211', '93495', '45576', '70614', '27988', '53210', '18618', '21318', '68059', '25518', '55917', '56522', '16548', '2404', '93538', '61452', '66358', '3709', '23914', '92426', '81439', '38070', '28988', '29939', '2948', '85720', '45628', '51101', '89431', '86365', '17571', '50987', '83849', '11015', '83812', '66187', '26362', '66786', '22024', '93866', '36161', '90080', '64874', '37294', '83860', '73821', '80279', '36766', '73117', '44620', '84556', '42070', '90383', '27862', '20665', '67576', '34997', '57958', '80638', '84351', '63961', '1362', '14338', '80377', '24192', '41294', '57368', '51189', '27287', '45764', '86289', '65600', '708', '84090', '96005', '55676', '84855', '72385', '70018', '9336', '82701', '3710', '52083', '74045', '96454', '30956', '67369', '78941', '81810', '71906', '23194', '33042', '50794', '61256', '24449', '48639', '22916', '78303', '13666', '40762', '43942', '51075', '89783', '95786', '90462', '6181', '36482', '40675', '4970', '6388', '91849', '72579', '94983', '86084', '20140', '68427', '48123', '43122', '98066', '37560', '6927', '72803', '5546', '62259', '98439', '6457', '98568', '70499', '33022', '28226', '29675', '20917', '75365', '20900', '8190', '56736', '99153', '77779', '49333', '50293', '97650', '4067', '47278', '42761', '71875', '13966', '11223', '46783', '18059', '61355', '29638', '75681', '24466', '89634', '20759', '83252', '37780', '15931', '74893', '6703', '64524', '80656', '85990', '78427', '18411', '20696', '86432', '93176', '69889', '15072', '15180', '9935', '10467', '60248', '42430', '62590', '89596', '27743', '26398', '79912', '60048', '50943', '38870', '69383', '72261', '98059', '55242', '74905', '5667', '54321', '70415', '39903', '49711', '85318', '79979', '59262', '82321', '15263', '17416', '74554', '94733', '72112', '49872', '54849', '73883', '78250', '74935', '68559', '57564', '50541', '45730', '41595', '5588', '83723', '42891', '11898', '14348', '99732', '14481', '85233', '21277', '94508', '52551', '74187', '7634', '42912', '25100', '43536', '35798', '48190', '86477', '22680', '48148', '59501', '56563', '16802', '81496', '97568', '68657', '51462', '67953', '99660', '39002', '54170', '57190', '68086', '52700', '6487', '55709', '70418', '62629', '70420', '35695', '36152', '45360', '53503', '46623', '76000', '50648', '97876', '44815', '29163', '1356', '64123', '71388', '17658', '99084', '58727', '59437', '38773', '71254', '81286', '97545', '18786', '56834', '20346', '36401', '62316', '58082', '67959', '99876', '69895', '80099', '62747', '20517', '99777', '6472', '49189', '31321', '39992', '68073', '13378', '51806', '21776', '52060', '96983', '25754', '93709', '96627', '8644', '93726', '14002', '37716', '87620', '34507', '76339', '24491', '5849', '44110', '522', '66521', '12776', '44887', '80535', '14548', '75248', '671', '73071', '35715', '59474', '7061', '82243', '56170', '20179', '59717', '1725', '24634', '11270', '77023', '63840', '46608', '44667', '22422', '59771', '94768', '73033', '82905', '16463', '40971', '22204', '58366', '28721', '14907', '76468', '81872', '38418', '36989', '61439', '10610', '131', '44296', '35453', '10117', '75856', '94603', '99602', '68075', '35949', '13599', '50030', '69633', '55956', '85465', '16429', '86081', '11145', '6195', '82207', '90598', '92814', '23725', '83204', '80346', '71542', '46634', '15820', '54123', '45397', '15322', '61743', '9273', '71347', '6835', '64006', '91718', '43677', '32923', '21486', '17098', '61694', '43347', '40019', '4071', '52443', '42386', '56839', '83514', '27633', '40780', '51749', '92101', '62384', '92206', '56044', '66174', '11137', '73966', '78471', '30468', '31643', '33197', '6888', '8066', '86603', '74383', '6098', '54411', '98819', '89862', '88639', '94422', '89371', '80526', '91747', '91220', '64944', '76658', '42046', '58518', '27249', '6646', '3028', '1346', '33763', '9734', '31737', '65527', '5892', '60813', '3410', '35464', '43009', '98382', '70580', '93898', '56404', '32995', '62771', '71556', '40538', '55612', '45656', '10758', '20268', '33603', '38310', '14242', '74397', '10722', '71575', '22590', '49043', '91439', '9055', '23668', '9101', '5268', '64133', '77501', '64684', '11337', '47575', '50732', '88680', '93730', '46785', '17589', '3520', '57595', '71241', '34994', '8753', '36147', '88844', '41914', '11250', '94632', '71927', '4624', '86279', '7664', '2659', '94853', '65386', '30438', '86005', '92883', '84629', '59910', '44484', '1306', '8404', '56962', '29990', '38445', '96191', '73013', '66590', '40951', '24712', '18825', '37268', '87843', '18972', '12154', '7779', '52149', '76152', '65799', '86011', '35475', '78083', '88232', '91551', '65532', '93516', '73827', '24227', '44687', '55759', '83819', '45088', '10856', '60488', '39051', '14103', '76650', '81181', '46731', '737', '58788', '78945', '42096', '66731', '66740', '72273', '88969', '5655', '86590', '41096', '80038', '32430', '51877', '23970', '91900', '13082', '45880', '94367', '19739', '61998', '71665', '16083', '57035', '26916', '10166', '18834', '46798', '66881', '28444', '68840', '10459', '81087', '4728', '76224', '39257', '23470', '93524', '37345', '30074', '49856', '22022', '55279', '5159', '5193', '58030', '57539', '12514', '49759', '96222', '52597', '67192', '88187', '53614', '16084', '79915', '28212', '79334', '85283', '32306', '31058', '43113', '74707', '74869', '2213', '32134', '6379', '85426', '87098', '35984', '51105', '69287', '16803', '83337', '14913', '62531', '58098', '7914', '20105', '28850', '1384', '43173', '62983', '87113', '76066', '86320', '77684', '45191', '95225', '41503', '36713', '48404', '91228', '53865', '98981', '59161', '61237', '84561', '17455', '14379', '57789', '80895', '99260', '84595', '72942', '53220', '84448', '81332', '49437', '83086', '93414', '54519', '52288', '74772', '22460', '49324', '11168', '96071', '61985', '38284', '6405', '54698', '71727', '60093', '37340', '87884', '83403', '4542', '94949', '19636', '15855', '39105', '10424', '67418', '91022', '69254', '8481', '38411', '3832', '44354', '93548', '57172', '28481', '372', '81497', '52179', '41060', '72141', '41396', '65590', '70432', '82819', '93814', '26118', '84780', '88485', '70821', '8222', '83000', '47067', '38516', '33347', '47681', '48202', '60749', '52112', '7937', '28105', '11394', '45746', '43252', '34494', '2979', '69715', '42486', '82315', '71760', '97413', '66137', '94487', '7429', '74434', '22964', '55251', '3448', '53534', '2574', '9693', '96157', '2955', '4348', '19566', '56930', '83319', '31310', '53905', '1148', '41726', '22233', '76045', '37351', '10545', '17581', '28047', '30199', '4741', '58111', '33497', '67796', '67730', '31247', '43772', '29461', '45970', '73353', '22534', '53962', '32147', '71392', '62579', '66345', '58246', '33442', '9581', '29705', '14058', '86471', '76125', '59363', '94982', '74810', '89149', '20066', '3366', '3568', '25752', '80036', '64119', '27270', '40061', '91052', '69022', '9852', '77112', '83075', '43924', '61661', '56133', '96652', '57944', '72576', '82170', '79236', '55745', '15309', '88878', '72761', '37647', '67465', '12777', '97309', '93202', '41470', '8787', '64920', '48514', '18917', '35157', '59151', '4640', '5317', '38134', '76548', '82788', '9214', '58418', '73185', '90554', '10543', '47182', '62936', '91765', '89751', '68931', '48865', '64607', '7150', '77862', '14297', '14828', '33013', '91698', '67593', '98096', '16595', '51639', '86531', '24719', '1703', '78788', '43810', '38918', '95491', '99903', '82671', '8291', '68288', '31224', '39863', '4265', '77798', '7698', '33804', '92286', '4744', '37038', '44203', '98212', '17369', '77442', '62879', '4145', '96881', '15646', '36824', '19959', '45451', '76049', '54272', '97577', '95298', '81115', '30204', '82041', '8037', '10052', '8756', '76833', '82851', '24276', '75574', '36037', '78079', '92807', '29064', '90000', '84150', '17102', '75092', '49424', '35597', '4693', '82853', '42511', '16119', '23478', '65240', '55585', '91762', '71671', '46682', '72479', '97696', '24615', '12579', '30274', '48255', '2336', '90202', '5808', '45426', '76308', '74639', '31245', '99894', '89638', '6233', '33893', '71899', '85273', '89429', '29761', '50231', '57249', '99347', '22642', '66972', '86221', '47514', '88274', '10819', '73150', '53754', '13304', '20478', '38099', '619', '14669', '8011', '97657', '26569', '65430', '13467', '38180', '23675', '72350', '42257', '39875', '23529', '53407', '11833', + '29599', '95621', '7727', '59527', '86846', '22860', '5358', '3730', '87555', '362', '95755', '54565', '29935', '68950', '52349', '98344', '86576', '7420', '12236', '15844', '48099', '97535', '97081', '50261', '31187', '60496', '24123', '24042', '6376', '6679', '99806', '20306', '60676', '36881', '77309', '5247', '96569', '53417', '73252', '64179', '35318', '75732', '65119', '32621', '40464', '22887', '96152', '65161', '83381', '8915', '68142', '7328', '85031', '15688', '72519', '93992', '86927', '75538', '38205', '50877', '70039', '97538', '94822', '52131', '49643', '85206', '1347', '14574', '88736', '53442', '49991', '64925', '72283', '82213', '60905', '36118', '62963', '16983', '79185', '15111', '26059', '17792', '98218', '33214', '1094', '41754', '77275', '65173', '13190', '91004', '90422', '44387', '92672', '98641', '54609', '83295', '37395', '70104', '32986', '72524', '82478', '5837', '83916', '52736', '57112', '55985', '42642', '42136', '89642', '35712', '49489', '19726', '65824', '24384', '48112', '15366', '99206', '68384', '51389', '529', '21475', '75749', '95182', '60110', '70571', '74174', '38105', '78107', '4101', '8982', '11215', '23987', '3303', '28706', '54629', '98000', '67510', '30036', '99140', '48896', '40971', '7735', '79984', '50134', '94928', '57023', '52880', '83067', '41940', '62994', '89213', '38593', '19283', '68206', '22234', '19245', '26266', '32403', '65889', '17022', '64280', '42797', '27161', '57675', '42313', '93606', '93082', '20659', '90824', '1226', '66266', '12503', '57104', '15247', '51160', '92398', '71967', '59476', '44465', '35765', '10787', '47737', '45792', '2292', '47599', '89612', '8162', '87622', '69410', '45727', '31158', '99791', '89544', '27214', '99588', '40516', '75616', '36505', '46079', '95448', '97999', '47462', '47799', '82729', '34038', '60789', '96938', '22682', '79062', '93307', '36038', '49016', '90983', '48219', '50889', '32517', '72219', '71229', '82643', '1195', '70543', '17', '22178', '23544', '72371', '1163', '28527', '7336', '39846', '31956', '80963', '41804', '59791', '41831', '1940', '52377', '79494', '12531', '81112', '44320', '18746', '5774', '63869', '4085', '59922', '12751', '99443', '13530', '23872', '36026', '83360', '32711', '92980', '11140', '99323', '57263', '98149', '29265', '25548', '65995', '4818', '15593', '8535', '37863', '12217', '14474', '66584', '89272', '86690', '58777', '39666', '44756', '18442', '52586', '98030', '40850', '38708', '49304', '68923', '65008', '84388', '83639', '29866', '63675', '26793', '49227', '82099', '24090', '57535', '24201', '65776', '74054', '89833', '62979', '26613', '5851', '99766', '63484', '66605', '37179', '90760', '59336', '58390', '93239', '84578', '11396', '93994', '73818', '23972', '37720', '72369', '25063', '32952', '71036', '76612', '31285', '34090', '19136', '53783', '66436', '61478', '96749', '43658', '7399', '31574', '67073', '40480', '20727', '70993', '65549', '30800', '21507', '53785', '89574', '86381', '56492', '62603', '44856', '68687', '63794', '70996', '7475', '84238', '71939', '86886', '94792', '15036', '36936', '95722', '17771', '67850', '33371', '49314', '40744', '5432', '81057', '41201', '75986', '22961', '15323', '1570', '18657', '95219', '19130', '53127', '15867', '81135', '73206', '76668', '36386', '48828', '31417', '56916', '70891', '60534', '95777', '10022', '94053', '2928', '56326', '16559', '79656', '6414', '81247', '78270', '55687', '19151', '61597', '99857', '81142', '27725', '53493', '12185', '1455', '48501', '59425', '20591', '24900', '66079', '84889', '32024', '18919', '2043', '7076', '71201', '88258', '86521', '93348', '26395', '39646', '44145', '33911', '46231', '67054', '39979', '11630', '23020', '76278', '88056', '11480', '4723', '78612', '70211', '60622', '84687', '59092', '65675', '38479', '64399', '64699', '95964', '42764', '69060', '28189', '4193', '95805', '75462', '17245', '59640', '94773', '84292', '53092', '98507', '61353', '32483', '53027', '48912', '87221', '47788', '59263', '65196', '35567', '17494', '64253', '50223', '7057', '87467', '62414', '2523', '50910', '72353', '78986', '78104', '47719', '29108', '12957', '5114', '64435', '66707', '37449', '70399', '45334', '71606', '55338', '55072', '58765', '12151', '22012', '16954', '87366', '14240', '98041', '72296', '47408', '56879', '99584', '63172', '92316', '28071', '29880', '19608', '13839', '87484', '56541', '88662', '87098', '72124', '78282', '27653', '38993', '31870', '67239', '99445', '7376', '78487', '98880', '12180', '86773', '67773', '15416', '58172', '13075', '67559', '97510', '29705', '86985', '57024', '11827', '31236', '91920', '26116', '94614', '14486', '46252', '78847', '43786', '70048', '96739', '35240', '39933', '58209', '27852', '65669', '47323', '58150', '84444', '44344', '95882', '41258', '31314', '69060', '19916', '6979', '19436', '45572', '16259', '74566', '6306', '24705', '53422', '593', '97031', '22308', '26875', '23042', '78035', '34229', '61976', '23175', '50072', '90896', '50810', '71730', '86468', '94807', '8218', '36032', '58628', '60560', '51206', '37943', '27987', '15014', '49905', '70018', '66799', '80851', '23594', '29982', '6438', '97381', '47715', '96294', '17985', '48545', '12672', '5250', '9988', '24601', '3736', '97815', '54363', '64703', '44167', '68376', '16595', '38073', '29630', '59630', '1858', '71823', '75580', '70083', '14493', '93821', '93394', '85369', '3818', '8435', '59988', '43966', '13961', '15855', '83332', '80312', '27299', '88840', '76964', '56173', '62794', '79389', '82642', '85843', '47116', '43064', '16061', '28905', '54415', '72832', '91252', '93488', '79457', '99336', '70744', '80432', '6487', '880', '87701', '154', '86574', '86677', '17892', '81488', '95260', '12515', '43189', '9211', '55403', '41417', '60046', '54785', '83655', '28274', '65745', '63062', '44549', '36391', '48051', '7328', '3572', '33226', '49177', '25123', '59065', '19691', '15109', '10172', '95578', '29497', '48152', '20276', '36270', '78866', '48309', '53209', '55475', '30073', '19717', '16004', '45692', '83430', '9291', '45935', '57030', '92613', '91656', '67697', '34915', '28156', '56594', '3273', '11194', '98270', '34370', '2621', '66679', '97451', '97717', '87923', '48310', '37725', '69743', '75103', '84956', '75163', '16069', '65304', '19397', '18071', '27273', '49823', '57595', '98324', '82174', '10293', '80943', '64184', '19472', '4198', '9410', '25927', '65961', '33155', '95168', '33692', '61712', '69877', '13308', '17415', '10022', '2491', '67310', '96140', '68050', '76272', '17143', '76805', '57176', '7539', '22690', '95483', '87592', '27221', '90821', '51154', '99828', '68998', '54581', '74222', '10269', '65057', '45467', '96089', '55058', '89779', '60837', '74122', '52886', '58055', '14880', '93208', '66652', '68830', '24121', '62407', '87257', '18802', '14925', '45423', '98624', '55195', '59072', '41414', '77840', '66075', '62705', '26549', '19063', '57552', '2507', '52069', '57620', '66688', '14833', '33700', '90666', '98052', '5367', '2268', '43093', '69063', '22030', '85564', '92258', '1847', '24446', '65835', '38660', '91899', '87732', '52396', '31952', '36000', '86944', '16109', '80729', '53757', '60226', '59103', '84187', '36674', '72823', '29884', '4654', '69139', '20440', '57413', '3651', '39639', '44564', '57492', '84159', '751', '99748', '9659', '72661', '39220', '99742', '74734', '75729', '38071', '69934', '73640', '65294', '54524', '64372', '37927', '17187', '7863', '12732', '40296', '36197', '15821', '76831', '4400', '71933', '4040', '22072', '33064', '25702', '13324', '91275', '27388', '97729', '14620', '45989', '80737', '17934', '4219', '3032', '43457', '31051', '24469', '67041', '29328', '75499', '80951', '88212', '92595', '49969', '24612', '58732', '2718', '3805', '50918', '99426', '8614', '35580', '93273', '989', '24385', '41185', '25687', '47146', '25227', '95839', '56355', '98536', '79824', '31725', '46447', '26690', '68418', '47783', '33725', '21729', '70797', '59038', '60376', '25087', '68332', '67950', '12411', '95918', '64736', '65336', '74947', '64605', '4106', '42712', '96640', '28492', '28648', '42429', '821', '24333', '69677', '38959', '23484', '92005', '29352', '29159', '52873', '99947', '21834', '85347', '93479', '28298', '55608', '3226', '69714', '80283', '6577', '18849', '44605', '75286', '28139', '26541', '12867', '57500', '86617', '33005', '57498', '60223', '74954', '51401', '55246', '5648', '16513', '40930', '43821', '32090', '66002', '65530', '76083', '6047', '6879', '94987', '80787', '11688', '77161', '92670', '6696', '400', '28572', '47234', '51375', '88518', '762', '92617', '54260', '7560', '60180', '43331', '64059', '27616', '75839', '21392', '47756', '46254', '19486', '88533', '30130', '93694', '8557', '66534', '94447', '16910', '6480', '77440', '24366', '6195', '48946', '28597', '44429', '50300', '73556', '40638', '98709', '94413', '15987', '43860', '64871', '93953', '34506', '7296', '31753', '30626', '77510', '39829', '25696', '39776', '69185', '36540', '65413', '31528', '43446', '73532', '49776', '30282', '30004', '26725', '15200', '33958', '90320', '71836', '48051', '31970', '5326', '96194', '69695', '60898', '60945', '18271', '50868', '61468', '23593', '68985', '20628', '58044', '8942', '34849', '7384', '50500', '62895', '78780', '48946', '65278', '4067', '973', '34761', '15512', '73739', '23138', '47322', '55568', '32259', '71816', '49277', '75218', '76104', '19579', '68312', '67904', '33886', '53888', '26421', '43859', '40291', '39068', '31711', '36542', '10195', '39781', '72352', '13188', '34113', '9428', '60443', '4987', '13783', '80744', '63483', '18266', '11961', '87167', '46987', '28480', '74214', '39191', '8146', '38090', '75727', '79245', '47720', '52547', '45321', '4972', '49701', '74354', '69672', '63455', '41902', '5667', '54166', '4962', '25873', '44509', '73332', '73383', '29438', '21455', '12320', '11997', '16921', '49379', '63027', '86175', '8110', '76149', '2520', '11256', '25863', '50518', '69001', + '79113', '9447', '91840', '5242', '10998', '46496', '2448', '56058', '20970', '10517', '17783', '25723', '97137', '62840', '1264', '78691', '81020', '55335', '48524', '2088', '90413', '76651', '26855', '16177', '14954', '62914', '21344', '5708', '75560', '39311', '95865', '28783', '64902', '95657', '46276', '33426', '4799', '11588', '57513', '73689', '77677', '63011', '97795', '34954', '76866', '32043', '32697', '26643', '36890', '53476', '3011', '13963', '49551', '87671', '67761', '17488', '94770', '50599', '33272', '23091', '38079', '41177', '22395', '91656', '79679', '38687', '57384', '80118', '42507', '4098', '78949', '45669', '48802', '83915', '78292', '4369', '57657', '49146', '45192', '98491', '72457', '46331', '207', '81601', '7409', '70856', '91605', '70295', '9171', '72293', '32997', '78025', '16795', '73534', '68780', '21284', '31767', '94381', '86439', '12420', '53285', '99563', '60502', '67954', '55012', '99809', '5431', '69978', '99712', '14401', '79498', '4495', '3045', '528', '72542', '91604', '72725', '39378', '80378', '41996', '20138', '54545', '59730', '36951', '45157', '37964', '97690', '12184', '4944', '53803', '93605', '60851', '68938', '46285', '89663', '90309', '6907', '87239', '81791', '83292', '90013', '68927', '14725', '81840', '63836', '52068', '43830', '4794', '931', '59255', '8263', '99057', '94401', '69033', '7437', '20364', '92884', '28193', '43932', '37629', '59426', '18891', '8583', '79551', '87242', '1483', '6725', '65786', '16844', '12650', '99305', '42841', '9811', '18800', '39313', '51373', '31874', '84558', '27831', '48614', '48975', '55509', '83363', '31854', '64001', '94028', '76125', '79314', '24893', '81132', '9441', '86015', '28356', '40358', '10160', '23328', '7330', '76538', '37611', '89351', '84132', '97047', '26109', '95222', '35130', '75600', '88602', '15073', '87835', '71649', '28948', '81615', '37498', '28674', '59776', '44095', '65924', '64368', '94536', '12518', '61711', '55619', '82949', '4114', '21540', '70544', '28022', '79983', '28781', '7749', '97873', '4951', '50076', '47611', '99522', '56820', '38653', '49047', '36283', '83908', '72452', '85625', '10811', '36998', '44083', '34864', '44975', '39057', '4551', '68450', '24781', '1503', '9871', '46885', '11424', '21259', '54900', '97669', '85669', '6015', '2521', '37661', '14915', '57423', '91903', '94789', '32059', '64972', '4600', '61465', '27118', '79785', '13547', '49766', '38410', '68860', '63756', '23621', '64387', '46255', '63408', '11297', '41081', '56326', '58349', '98703', '72268', '73574', '32098', '42534', '91502', '38083', '11241', '56828', '12098', '25377', '37054', '56328', '30034', '26922', '68401', '93478', '63275', '62650', '81407', '773', '79499', '14970', '47217', '1187', '57428', '69980', '77764', '74791', '22107', '54363', '39247', '56028', '56982', '84244', '21464', '18716', '25533', '94589', '94768', '21537', '18436', '81135', '27654', '79713', '56630', '61571', '58453', '26758', '68450', '68449', '2994', '15347', '83954', '71823', '6428', '44210', '79597', '95144', '32871', '1991', '320', '77157', '63607', '31154', '48846', '71125', '61750', '59608', '33038', '35733', '68915', '94127', '50383', '64242', '49708', '57270', '65019', '8581', '12111', '18487', '50013', '58664', '22214', '19033', '33681', '44754', '28830', '10381', '52318', '34959', '20682', '55453', '53800', '65774', '99164', '72102', '36986', '44157', '56716', '7974', '81475', '25926', '39402', '33688', '99671', '95312', '42268', '26536', '14482', '67377', '57993', '89147', '15834', '64995', '4700', '18714', '30221', '39095', '32749', '69257', '55204', '30497', '31839', '63045', '30009', '62683', '31232', '77680', '93551', '63589', '6989', '77246', '42169', '46117', '73226', '37427', '1858', '83649', '37410', '86369', '4641', '74481', '66168', '48041', '22597', '14670', '27464', '57165', '20939', '36282', '76940', '73358', '50521', '69603', '8895', '81793', '57743', '81903', '64025', '91641', '25276', '34040', '62642', '64015', '57657', '84890', '73832', '782', '60160', '16998', '40023', '24590', '88613', '76640', '53091', '67600', '80183', '45674', '64464', '25163', '42384', '66972', '13953', '41966', '66048', '15135', '73745', '19466', '53657', '34619', '13462', '15905', '48257', '73297', '238', '93525', '80556', '5942', '5411', '66169', '9090', '95130', '74316', '57321', '48083', '62355', '68113', '15239', '36644', '80326', '65817', '54428', '61955', '58849', '77206', '16073', '98261', '92091', '39178', '35464', '85109', '85452', '21128', '25665', '81860', '44664', '24024', '56960', '95124', '39786', '18836', '11121', '44163', '81074', '79064', '46219', '94694', '44233', '81469', '24642', '15030', '21995', '13587', '40755', '6669', '81093', '74305', '1881', '55649', '37273', '80827', '98643', '46694', '59281', '79231', '42813', '84984', '7052', '98113', '17296', '84434', '31205', '46894', '71219', '74530', '44686', '70744', '91388', '20692', '96853', '73803', '15836', '18126', '49686', '4179', '47588', '87892', '65425', '68012', '97468', '92510', '99271', '58694', '11918', '37051', '18644', '57228', '14265', '57572', '57022', '52186', '30193', '93570', '87872', '5257', '26784', '6476', '61746', '68559', '1720', '26202', '16519', '27688', '10645', '87174', '60845', '73385', '82075', '6933', '98828', '56895', '17344', '84253', '36561', '51648', '24939', '63470', '31034', '95052', '51090', '51465', '87979', '68650', '30181', '29598', '19137', '43221', '81353', '90170', '96985', '61115', '17385', '92314', '80650', '55821', '17874', '84333', '93272', '48260', '87272', '22764', '59957', '51870', '85988', '39222', '77241', '62535', '28344', '6011', '80831', '64551', '46299', '75195', '71177', '8660', '58943', '57003', '3306', '74413', '74068', '15073', '89016', '93140', '13911', '57170', '19880', '41870', '9131', '57495', '73032', '86979', '60094', '87026', '30880', '4736', '86301', '92707', '21689', '83565', '71275', '47665', '65687', '71184', '89897', '32490', '97577', '38723', '79113', '37531', '97500', '94450', '15699', '58019', '84423', '27057', '56017', '97148', '47365', '30669', '33818', '80406', '99690', '33012', '95178', '46809', '48448', '79350', '9146', '99701', '98976', '71197', '44161', '75069', '36602', '79650', '97301', '12020', '56658', '25701', '46392', '78609', '63073', '69419', '57736', '20102', '42415', '79044', '20277', '56280', '47903', '94311', '25558', '40336', '91305', '90505', '66769', '64562', '83737', '62892', '10375', '71024', '19988', '56946', '76110', '21847', '43162', '50578', '46086', '54167', '61722', '53463', '63134', '69288', '12838', '14116', '71687', '50846', '59810', '24826', '84138', '82885', '91496', '98600', '82769', '40049', '4125', '50694', '1294', '2805', '29691', '82321', '76462', '85945', '115', '29188', '66918', '71340', '31585', '61638', '95472', '52978', '50622', '81990', '60955', '70519', '22270', '35610', '95871', '89222', '41038', '52546', '1163', '67943', '1793', '92010', '35755', '74509', '66665', '95759', '8568', '44299', '67822', '5806', '85839', '13895', '87675', '31357', '88014', '40026', '53050', '28951', '31992', '42495', '82892', '51567', '2869', '45808', '20238', '20781', '56098', '66307', '95701', '614', '60833', '3091', '81339', '24195', '65639', '85976', '28116', '66224', '51502', '73637', '13207', '88302', '36488', '65518', '98187', '26', '74367', '64706', '53943', '86760', '25783', '82112', '34958', '86621', '20848', '63459', '14049', '84943', '91873', '50238', '77773', '64109', '8602', '87934', '47583', '66053', '30287', '5507', '80312', '37464', '57457', '86200', '17806', '16522', '38843', '94334', '59958', '63864', '53427', '74506', '33980', '90449', '30842', '53616', '36738', '52', '13595', '53051', '13174', '60163', '71420', '73835', '67119', '79018', '42782', '45059', '952', '46360', '85879', '71552', '84741', '29746', '32577', '10041', '7208', '97528', '51256', '916', '55973', '17684', '99046', '38782', '58660', '97798', '66032', '48339', '51329', '12532', '97904', '95454', '42737', '62541', '96702', '82953', '94610', '26645', '86813', '25480', '99713', '26078', '23028', '93056', '21445', '73209', '89318', '69987', '34705', '30064', '17094', '51135', '54141', '26625', '1086', '13082', '30843', '98672', '56864', '42605', '5833', '60850', '69366', '27351', '16456', '92609', '48030', '54322', '69891', '46502', '34578', '77918', '63276', '75958', '42519', '60266', '85576', '4855', '14258', '67017', '10545', '35078', '53012', '71922', '85784', '73402', '74363', '58457', '94102', '23510', '51559', '39482', '87057', '9377', '10106', '82985', '33931', '16523', '6484', '97749', '83172', '53753', '27466', '23073', '96083', '67302', '57465', '21877', '18013', '99804', '32873', '43123', '72365', '53197', '80578', '69770', '97471', '86954', '67183', '98497', '78474', '28450', '63183', '98699', '42738', '61433', '3491', '27304', '49311', '94980', '92740', '43272', '86549', '11406', '79636', '85582', '38086', '657', '2354', '26567', '77450', '42086', '21600', '49011', '44059', '47872', '75761', '96577', '11642', '83471', '79616', '23749', '77082', '96876', '65302', '84027', '48955', '59887', '20657', '75090', '9058', '50347', '66088', '70745', '76342', '58026', '95568', '61504', '93473', '84590', '47089', '74717', '93090', '46334', '68273', '59500', '54345', '72608', '54048', '86156', '40296', '74046', '6813', '36369', '74543', '18305', '85236', '31316', '37061', '96893', '23112', '5529', '10166', '19037', '1467', '70810', '30932', '18410', '92837', '81324', '12268', '54705', '25207', '90366', '56528', '3392', '88747', '39951', '97957', '99404', '23685', '13533', '15640', '11434', '66516', '71025', '65770', '88000', '52232', '32360', '10787', '37438', '2264', '94460', '80214', '42288', '59062', '29010', '64093', '21225', '22297', '36935', '19202', '5925', '85373', '27414', '28991', '9191', '42273', '56587', '89719', '77191', '64334', '61542', '28763', '28978', '79184', '59815', '95200', '30246', '54022', '287', '91808', '66347', '50833', '15356', '78614', + '65087', '9664', '67555', '58222', '20787', '72382', '83712', '15858', '72040', '59451', '7428', '47586', '83509', '45969', '28759', '94033', '88618', '46', '1753', '28290', '71934', '99070', '52517', '65865', '15904', '68134', '87595', '39779', '99174', '13366', '50847', '69037', '77911', '19709', '84404', '24370', '18920', '1746', '28059', '4653', '6892', '25496', '48810', '82943', '7069', '44018', '1617', '56497', '27582', '29576', '47883', '8385', '54975', '96245', '77257', '31194', '34512', '19198', '50926', '50886', '91577', '49260', '49604', '58464', '18198', '90887', '15379', '74883', '36644', '63967', '34545', '20539', '12884', '6010', '98636', '94832', '47702', '98318', '6078', '42517', '97030', '93707', '39286', '10245', '66230', '99574', '74563', '81937', '22153', '10659', '67579', '85147', '69686', '32017', '8199', '20027', '64266', '60707', '55426', '22027', '13823', '81404', '86757', '3754', '197', '45469', '68736', '17656', '92453', '6784', '35531', '33538', '27558', '35825', '13471', '16377', '66978', '44601', '69943', '84635', '7335', '67725', '88064', '78201', '64380', '66771', '62221', '60427', '32028', '37857', '5491', '2757', '10537', '2199', '60244', '14726', '29734', '34433', '81241', '39778', '65831', '56153', '81696', '25113', '17371', '79022', '3587', '60439', '96582', '37801', '82496', '68242', '19561', '53140', '17171', '13165', '58750', '80612', '19818', '16753', '70009', '75290', '48846', '89851', '25190', '35525', '84348', '65298', '23161', '60583', '89611', '18798', '85422', '60713', '17660', '41915', '65757', '4781', '48374', '10712', '40330', '2409', '41020', '28042', '51857', '81224', '93917', '90366', '58848', '8012', '92070', '75349', '80066', '83228', '55170', '78376', '94127', '29418', '58660', '74006', '63909', '59502', '63255', '32318', '28904', '26117', '88103', '50656', '58517', '16372', '43392', '77876', '65997', '63501', '45927', '17444', '84021', '54886', '57082', '47456', '73816', '66900', '82958', '49722', '95843', '23458', '4560', '67276', '59080', '81486', '2967', '26133', '87694', '74183', '22100', '45306', '33895', '26512', '71426', '54994', '47367', '10171', '45681', '36268', '3166', '2179', '64197', '90037', '4641', '96073', '72989', '55123', '48825', '2862', '66974', '30115', '41554', '26727', '54025', '69504', '38284', '7816', '36370', '75764', '69774', '43208', '43598', '39461', '21655', '12169', '77790', '44578', '70959', '13500', '40823', '52952', '88095', '82193', '66424', '79814', '25651', '58125', '42724', '11752', '64533', '34827', '72755', '96727', '82066', '60268', '55535', '4230', '73018', '69602', '16912', '66088', '21859', '38654', '38638', '51444', '37999', '18595', '15209', '21801', '21130', '53741', '54954', '91590', '83660', '74071', '82627', '4311', '45268', '94735', '65747', '72428', '22621', '70828', '64394', '647', '23570', '17677', '84377', '63993', '10268', '47426', '72750', '83772', '29090', '73284', '36683', '91143', '64546', '99836', '21999', '61793', '41557', '16714', '20586', '14251', '56633', '46146', '84730', '5445', '8367', '52621', '79252', '89737', '18431', '32134', '86975', '58939', '63385', '8504', '25998', '79549', '70558', '4870', '57604', '48374', '32388', '97871', '75590', '10919', '43825', '82858', '25807', '50366', '2841', '2716', '51315', '79719', '33159', '63382', '11327', '65184', '87243', '17850', '40149', '12171', '48486', '26380', '85432', '35427', '59256', '57601', '29529', '20553', '3006', '69997', '87919', '47626', '96026', '37445', '51735', '89296', '26702', '22837', '61279', '85193', '30624', '10676', '83075', '21477', '54708', '47485', '80297', '86204', '25192', '55197', '81177', '70551', '11003', '7001', '55017', '487', '40926', '99463', '79514', '69697', '52854', '88640', '44464', '66314', '48240', '49787', '81333', '14914', '12919', '51037', '88609', '25572', '61019', '85184', '12051', '61246', '52108', '98895', '24549', '86944', '73303', '9012', '25151', '97424', '43531', '44790', '63034', '86713', '76483', '17499', '86891', '3267', '45091', '39126', '94419', '35750', '35239', '49413', '62381', '92010', '7246', '26997', '47446', '26930', '22668', '43961', '51741', '63058', '41977', '62933', '9197', '94303', '29044', '53755', '22391', '85373', '66572', '74650', '7824', '49712', '40021', '99958', '60404', '37370', '89264', '52343', '90901', '44730', '4252', '5811', '22283', '25102', '86636', '75509', '47088', '62971', '85519', '7495', '16798', '25708', '59258', '89361', '43455', '86768', '29961', '4362', '60956', '81059', '87604', '46241', '98362', '21030', '76182', '83545', '59150', '99333', '68271', '32629', '3364', '96467', '40427', '307', '84267', '239', '11711', '40933', '24518', '33854', '60291', '94179', '5564', '33196', '73818', '22201', '97078', '69360', '94513', '42623', '6510', '67378', '79468', '10394', '96430', '87701', '2667', '64162', '93266', '48941', '55345', '56127', '22704', '8793', '57008', '36920', '80845', '81203', '98715', '84158', '49194', '72718', '52919', '4951', '8715', '82384', '8570', '26733', '62399', '27297', '79038', '57916', '46155', '82381', '1027', '20541', '40580', '44836', '17336', '55548', '87454', '63549', '61401', '88819', '23326', '37546', '49089', '50707', '23099', '20724', '67184', '3394', '95941', '76045', '20523', '30393', '14157', '69381', '94104', '15406', '46707', '36234', '9136', '96805', '4904', '20310', '77164', '15400', '87198', '48310', '59729', '35230', '61503', '42015', '86910', '23264', '23542', '49105', '58334', '52855', '78865', '73516', '43820', '96593', '33168', '26170', '88241', '4016', '79862', '39617', '95329', '57881', '71750', '89690', '22519', '59367', '19825', '12638', '36100', '14355', '79540', '8411', '94571', '80462', '11953', '55629', '14598', '81066', '48968', '74149', '36938', '4564', '59157', '75517', '14309', '64905', '59893', '62803', '6968', '33003', '40894', '48816', '77710', '16692', '19135', '98381', '49397', '24048', '17804', '26829', '24046', '44546', '28174', '2639', '27957', '38625', '85902', '9908', '77666', '14096', '89851', '23600', '20717', '32860', '44070', '67853', '85907', '23761', '789', '73370', '15649', '26472', '9174', '20081', '49415', '97463', '14565', '19834', '94738', '68045', '67542', '3552', '77093', '99071', '16832', '95177', '49571', '85821', '9368', '85170', '58449', '89633', '99302', '94158', '50274', '50223', '51128', '52847', '53598', '47826', '2883', '28192', '84324', '9530', '14385', '37292', '74795', '16821', '5010', '77101', '96596', '72384', '41399', '83046', '37927', '48213', '84574', '55438', '79315', '71005', '45190', '56133', '14016', '92180', '28365', '34307', '90665', '74758', '45080', '38017', '80599', '91174', '47907', '19299', '9707', '92260', '36784', '55855', '25939', '74472', '47402', '12352', '75591', '31459', '76711', '57940', '95535', '55231', '80645', '84613', '90280', '3500', '83695', '42992', '77226', '25326', '8093', '49838', '6457', '44814', '82819', '94986', '81818', '65186', '21378', '49711', '66248', '72391', '65826', '21284', '41399', '72858', '32223', '35915', '45221', '36871', '16193', '2522', '52963', '24506', '98239', '56752', '15192', '8451', '62802', '54777', '67983', '82177', '7303', '71445', '40559', '58655', '73468', '40491', '52864', '81558', '36789', '53134', '38801', '90600', '36385', '90996', '20526', '13627', '62253', '96989', '88864', '11928', '14681', '77082', '95659', '47182', '16085', '26894', '39758', '32803', '26458', '65779', '66110', '38888', '15490', '48120', '90238', '8736', '233', '76851', '86938', '8873', '95387', '33400', '50788', '13568', '43139', '29821', '70427', '21022', '93285', '57243', '51675', '77459', '33170', '81485', '85303', '44133', '203', '44175', '89151', '26705', '32990', '10885', '10208', '58477', '97729', '699', '40581', '8065', '95864', '89797', '47506', '19160', '85728', '58599', '96271', '75849', '30736', '18808', '25828', '52011', '66179', '70244', '81312', '19570', '14287', '61078', '81922', '63361', '80858', '306', '22156', '82923', '53176', '24864', '49193', '2961', '96536', '62449', '13213', '85347', '11442', '76615', '43037', '86122', '99676', '74309', '46017', '16175', '12685', '53550', '19080', '37283', '44921', '34991', '4433', '99561', '38215', '35662', '13892', '7814', '41718', '41231', '85589', '74048', '72169', '26946', '61732', '9135', '39009', '63534', '4114', '40417', '83412', '22562', '65546', '720', '90064', '73027', '50736', '71086', '12445', '3681', '65140', '56291', '47452', '48916', '65136', '53941', '15484', '21038', '47251', '78461', '31031', '4838', '87483', '732', '16111', '68263', '4811', '65979', '44283', '51659', '29997', '11280', '3880', '12399', '63450', '93862', '50981', '62790', '76784', '76696', '40906', '49261', '35863', '43747', '65954', '36125', '73713', '98678', '13409', '38773', '59417', '9763', '429', '66203', '7350', '32754', '36079', '72447', '63925', '12958', '65292', '80270', '78432', '50925', '74946', '7649', '13689', '68001', '52318', '5804', '51584', '63637', '85697', '50929', '25279', '20963', '83038', '31199', '64189', '17152', '20459', '66108', '66005', '53159', '53971', '32528', '77145', '52981', '88941', '68593', '86343', '97770', '25625', '17598', '55249', '89640', '94162', '65874', '36511', '21906', '70401', '30914', '24085', '42945', '55597', '78393', '98120', '38283', '44967', '76077', '45739', '92432', '11846', '59131', '26322', '82651', '94792', '37726', '23301', '11275', '98813', '34229', '65440', '51369', '55580', '35207', '84314', '35696', '50231', '9399', '18679', '27219', '22519', '26725', '63451', '79357', '55915', '77027', '30354', '45890', '10745', '60035', '7914', '43562', '12219', '30220', '29382', '72520', '87475', '83634', '15163', '32468', '72904', '18200', '84666', '43560', '58483', '64613', '52170', '55490', '45954', '99323', '26401', '61210', '10450', '70360', '42906', '32851', '70022', '49966', '62331', '84249', '98093', '62146', '88171', '73246', '48314', '45730', '56348', '59081', '16642', '57872', '81564', '32305', '4158', '47263', + '97234', '74415', '36159', '30014', '22241', '77574', '27990', '67395', '81471', '18512', '47546', '23386', '6156', '72893', '15332', '57667', '92657', '35148', '45316', '90482', '93819', '38065', '74899', '10850', '63285', '50968', '88081', '42486', '56089', '43288', '11239', '55127', '26637', '85974', '93879', '78540', '42451', '35082', '93809', '63793', '51039', '13518', '20239', '26933', '19322', '26648', '83711', '93163', '95951', '77389', '43771', '36873', '66976', '38082', '59476', '53984', '6951', '42615', '80733', '1120', '80595', '10500', '67420', '38885', '8892', '36205', '17170', '46564', '66415', '36372', '68305', '41470', '38699', '60900', '23468', '6927', '498', '55896', '4005', '50692', '67984', '91012', '97314', '94177', '57676', '36734', '32671', '44733', '34981', '52186', '15599', '75642', '82652', '22773', '97106', '20691', '38085', '50368', '74916', '76281', '54002', '85721', '99186', '78438', '27865', '34827', '61315', '8817', '93105', '71539', '35484', '58033', '17540', '32806', '87253', '15780', '80123', '34241', '70383', '95515', '27998', '47682', '32682', '93658', '15116', '93030', '71866', '35516', '70161', '7556', '64122', '54622', '14312', '64375', '28863', '27886', '34040', '10010', '1460', '58765', '71496', '37785', '5616', '84850', '2194', '55514', '33636', '7138', '36937', '44763', '61308', '63574', '8069', '24109', '17821', '82144', '53309', '26183', '84238', '36684', '45285', '41353', '29380', '93726', '59560', '96422', '34218', '66851', '49001', '4855', '46849', '19877', '76540', '61415', '32053', '38801', '74623', '92808', '6660', '85031', '97706', '34417', '51085', '72292', '30903', '45126', '69978', '11572', '68622', '64123', '59344', '67118', '46014', '59248', '69321', '31897', '72232', '48016', '36455', '87125', '85356', '96888', '28316', '23149', '34823', '29944', '18522', '14045', '973', '49472', '89165', '84112', '94087', '47881', '87600', '86330', '13384', '4528', '94572', '30634', '86728', '48525', '53243', '35911', '96382', '22525', '89914', '52213', '23007', '54827', '35721', '49121', '40203', '99040', '55039', '94543', '85440', '75098', '12566', '15579', '40603', '51321', '28805', '46550', '42499', '69794', '51322', '37286', '72035', '15494', '39787', '97639', '24343', '36299', '72759', '96672', '58801', '57467', '97185', '19493', '4431', '33940', '950', '46713', '12503', '35185', '31106', '93821', '1086', '31767', '17484', '52751', '70411', '12196', '502', '45391', '70720', '27507', '1282', '18633', '45031', '10475', '83570', '29539', '16753', '63494', '17010', '76069', '38913', '53948', '34175', '36761', '37575', '73655', '76272', '1069', '74456', '10440', '65502', '95286', '39384', '83461', '76477', '4167', '46503', '22373', '24523', '46024', '28959', '83219', '27807', '80905', '88626', '70234', '86888', '60954', '95397', '47973', '47944', '30154', '42698', '15144', '53223', '21123', '19281', '62933', '24952', '84744', '58456', '38446', '26768', '40797', '107', '52101', '8348', '8531', '17772', '97826', '34009', '15825', '48977', '54939', '78886', '31784', '42842', '22773', '9095', '25113', '65362', '83953', '55921', '92797', '26776', '59312', '56428', '52218', '50141', '52535', '22564', '50708', '50384', '16261', '529', '31406', '46452', '30954', '5385', '60190', '38790', '65440', '11891', '2196', '83560', '68884', '72400', '47795', '7501', '8916', '63267', '35567', '39889', '61818', '94343', '46612', '75029', '46988', '12411', '6123', '60904', '43704', '31143', '72544', '45919', '67953', '97776', '49517', '60476', '54437', '46261', '99778', '18541', '77626', '12537', '68459', '12688', '36284', '54901', '89375', '10538', '38415', '33160', '78777', '33532', '82538', '21329', '34359', '29544', '57035', '61945', '49098', '32484', '68236', '11088', '57533', '11044', '96786', '58861', '51390', '9582', '24229', '32604', '66195', '7311', '46441', '73383', '58033', '41866', '19293', '73085', '50627', '19818', '40498', '56436', '99865', '55516', '38917', '89889', '49571', '99408', '20144', '76122', '49593', '47906', '94763', '26488', '86343', '98334', '70112', '85244', '45622', '47828', '35306', '86838', '57186', '69907', '90594', '68206', '25118', '75445', '71715', '15800', '67558', '17783', '56525', '22970', '17354', '17139', '76827', '50325', '95314', '65432', '44045', '57928', '66247', '10673', '69744', '99485', '57467', '66889', '55246', '15095', '23540', '32890', '22959', '59472', '62237', '6956', '34669', '16547', '54868', '22598', '65120', '48525', '72750', '70559', '75706', '61831', '14032', '36146', '90211', '36082', '79712', '43018', '53903', '82490', '45698', '54202', '83079', '4867', '18044', '55144', '49189', '33278', '48387', '68822', '73057', '58118', '89796', '47318', '98383', '58759', '172', '56627', '79861', '86734', '25758', '80779', '55741', '40924', '24568', '23090', '62934', '94264', '58331', '18747', '15802', '81585', '92542', '97426', '18205', '11820', '62702', '16439', '18613', '45918', '47584', '98743', '39808', '33357', '16033', '87378', '87036', '88920', '99300', '89343', '7384', '36867', '36712', '96947', '92141', '85779', '81735', '86121', '19511', '70519', '62537', '10398', '99308', '63922', '45419', '17708', '2093', '85127', '55753', '97149', '82877', '64642', '30417', '151', '75464', '75992', '99440', '58040', '32083', '90604', '47830', '40087', '78899', '43638', '59066', '46579', '44828', '35130', '10670', '62051', '94127', '32024', '85455', '80408', '18295', '8106', '89380', '90499', '3196', '3826', '63462', '64317', '55570', '77162', '48564', '59842', '80241', '76221', '57327', '48049', '74336', '43168', '19559', '32059', '99226', '46384', '48111', '56013', '58953', '68426', '22', '72641', '16207', '595', '71706', '61213', '65921', '67517', '92618', '57660', '76414', '60998', '1829', '90176', '62932', '15052', '94997', '41', '58848', '56411', '18129', '86542', '16330', '2803', '5651', '47351', '57866', '3560', '97530', '8338', '59434', '50183', '39076', '2516', '16293', '6559', '31189', '83713', '58214', '32016', '56581', '46226', '74599', '33513', '51530', '20163', '56512', '1694', '8165', '17370', '2510', '46015', '80417', '77195', '69062', '717', '33699', '37638', '82238', '5436', '67991', '98554', '26668', '19565', '26500', '30133', '54349', '36668', '56773', '55187', '25458', '54126', '11833', '32007', '38881', '20038', '42186', '67850', '12001', '96089', '63397', '75035', '16744', '45208', '84186', '82044', '34818', '55858', '1623', '41600', '34753', '92242', '16416', '49800', '45309', '56520', '38893', '42865', '37476', '78860', '32503', '18086', '55004', '65795', '82926', '75725', '52917', '75463', '82202', '35158', '67065', '38454', '90106', '79681', '57317', '68357', '66487', '99640', '13182', '68684', '96739', '89887', '54307', '98357', '4597', '47645', '38560', '17334', '46291', '91096', '22675', '12995', '35767', '76173', '54754', '75206', '13345', '78860', '19929', '12740', '13007', '69737', '52454', '6784', '77198', '59565', '44548', '22924', '38773', '9522', '59392', '45244', '64754', '39640', '75795', '22725', '78314', '24738', '83167', '31049', '92122', '62117', '47314', '34032', '76951', '63164', '22515', '46057', '81625', '72164', '60013', '26171', '55608', '50486', '51422', '18277', '13682', '70130', '49689', '32278', '25430', '84394', '57371', '93398', '14139', '461', '37302', '25705', '25642', '70311', '81778', '66948', '51187', '99649', '44509', '12753', '39827', '57393', '40288', '26496', '1500', '32979', '62062', '5170', '25273', '26343', '21058', '66749', '52257', '26335', '4347', '82427', '51381', '66857', '58330', '15097', '23881', '93865', '97929', '11233', '67910', '22731', '70911', '93941', '46169', '31920', '16027', '1635', '16338', '49712', '29345', '26316', '15221', '54771', '34715', '35399', '80287', '90881', '33758', '46569', '52450', '40564', '34885', '5679', '53232', '85632', '87510', '4008', '87014', '49611', '94569', '96432', '57394', '66064', '34144', '67269', '83546', '5901', '25386', '57588', '55055', '6940', '8414', '31432', '75407', '86922', '33925', '24600', '48261', '94439', '34234', '69028', '37145', '32386', '68275', '13237', '3521', '57395', '55029', '16426', '87797', '18559', '72744', '1155', '30384', '19799', '50001', '85831', '19583', '83898', '13465', '58160', '51160', '89216', '90725', '37151', '32241', '6319', '95146', '10376', '55137', '64094', '14990', '19747', '85852', '50692', '63214', '53123', '97058', '60203', '29272', '14439', '69712', '13023', '75808', '67150', '76276', '54521', '46621', '78807', '89302', '30273', '85103', '63180', '75468', '98474', '37749', '66275', '61712', '39381', '2740', '83025', '18083', '42030', '3391', '56204', '3960', '34780', '22476', '50074', '79854', '89447', '3695', '2986', '85090', '71529', '77995', '51572', '60278', '69314', '70777', '46754', '22085', '94517', '98052', '5700', '43157', '12345', '34663', '40350', '5739', '84384', '8354', '81566', '75500', '88934', '67062', '52877', '30428', '95593', '13288', '51836', '25264', '19278', '81290', '47555', '7606', '46694', '29735', '25723', '81597', '26221', '24545', '98576', '63395', '36061', '58752', '8486', '97817', '27235', '18579', '2387', '88017', '42855', '5111', '49094', '28448', '82227', '63436', '89135', '86591', '2922', '72163', '83000', '14378', '81829', '5610', '93513', '78135', '55292', '80776', '21212', '40604', '44442', '50505', '22750', '47985', '19367', '63494', '4363', '64500', '79444', '36504', '43418', '72523', '85981', '13740', '302', '76267', '14962', '79643', '26144', '27125', '13364', '82935', '8936', '8340', '38025', '80742', '42829', '95357', '11131', '6806', '10497', '70527', '3004', '54217', '51051', '34722', '57348', '50489', '76514', '76546', '29498', '40587', '52697', '37808', '32509', '68927', '68051', '53328', '41926', '9344', '67795', '98626', '44058', '46817', '95773', '57539', '13352', '96292', '72647', '47111', '48595', '82684', '43566', '93381', '3081', '27653', '25045', '34753', '80870', '92013', '21935', '89454', '14127', '73278', '87015', + '65232', '97216', '26840', '21272', '8226', '42347', '44358', '26715', '7860', '31989', '14091', '15648', '85801', '20463', '85221', '77969', '87196', '28563', '69663', '55169', '11372', '38915', '74678', '1183', '14845', '85696', '54995', '62342', '83227', '73536', '64251', '46556', '78421', '89058', '41036', '79746', '42148', '77720', '33923', '24979', '98428', '80202', '36845', '20584', '13506', '97375', '48434', '14863', '29237', '99014', '20650', '48496', '61825', '89100', '51501', '22632', '89030', '22901', '54072', '68093', '98140', '1631', '43266', '34532', '35780', '17763', '41731', '19580', '72266', '66530', '55650', '51506', '92256', '5332', '85143', '65002', '30530', '21358', '88420', '81349', '94086', '68143', '95343', '30490', '61605', '40296', '61665', '21032', '498', '67165', '3151', '94788', '36502', '77434', '88776', '67565', '4404', '51177', '49030', '88502', '92552', '31170', '92051', '79960', '43784', '17864', '54179', '91590', '39153', '75017', '11161', '31100', '27964', '30263', '54908', '57991', '89272', '14642', '40576', '20224', '48887', '93479', '87862', '19858', '78238', '4113', '32369', '70072', '82752', '85122', '72450', '53113', '53806', '45347', '55032', '77242', '70401', '83685', '41229', '47489', '36964', '40556', '2619', '64080', '7354', '10159', '74827', '11846', '64587', '88667', '20940', '93594', '43669', '38592', '20211', '96055', '65995', '89957', '18941', '82750', '90753', '49989', '63331', '52911', '1675', '98447', '35516', '28539', '91592', '80615', '68066', '63922', '52509', '26025', '14634', '69439', '19820', '89983', '38306', '84048', '66264', '26855', '3634', '92094', '61088', '28293', '11428', '68748', '10665', '27114', '20151', '65883', '86423', '77069', '74997', '147', '41427', '26124', '22163', '82452', '61186', '89370', '58583', '2692', '54969', '84679', '26237', '46229', '16041', '28473', '38684', '43587', '37886', '68790', '92030', '85066', '13673', '30280', '91844', '43115', '10005', '1481', '84393', '10008', '60291', '90687', '99338', '95570', '39916', '68643', '41309', '24724', '57188', '75473', '21647', '50206', '6820', '73756', '51102', '76169', '99840', '32863', '17306', '43063', '90483', '15894', '59042', '53797', '93961', '12248', '12125', '39393', '17056', '11576', '6145', '43960', '74245', '86805', '95665', '58855', '50039', '14867', '85076', '3939', '55501', '18184', '62019', '16312', '37518', '18544', '67431', '68250', '70766', '74635', '41835', '74538', '77822', '47918', '90863', '17526', '78930', '47473', '67945', '70385', '19404', '45030', '13476', '46186', '28191', '22019', '18700', '90915', '41408', '63333', '52902', '87336', '72833', '47707', '46692', '47873', '82012', '22597', '46379', '69109', '72108', '47439', '51130', '41894', '29827', '88403', '75040', '37504', '69925', '19034', '57998', '46633', '1021', '48358', '55923', '3536', '26350', '36696', '29658', '25247', '65614', '19042', '34463', '85938', '26155', '49801', '74604', '17581', '91632', '12935', '97966', '14947', '71196', '32270', '51915', '41692', '83274', '6608', '37190', '35363', '85313', '97545', '5696', '23700', '39334', '38434', '20192', '27349', '71340', '19732', '96142', '17152', '32995', '34829', '71157', '36376', '11522', '26646', '19239', '34033', '4200', '39023', '64291', '7931', '42807', '2800', '75490', '4515', '84571', '59557', '13421', '69465', '69121', '82358', '18134', '18536', '74294', '24013', '43777', '94192', '77031', '33341', '98070', '27534', '34419', '40107', '17826', '48196', '82843', '37592', '40484', '49019', '59822', '26118', '89788', '5990', '99595', '35178', '49473', '54699', '81618', '23497', '87025', '29421', '1685', '31406', '54115', '25193', '44525', '44886', '4885', '74482', '63280', '57532', '29773', '83104', '5354', '35313', '12558', '16165', '52712', '82282', '73541', '97334', '7283', '29405', '87694', '64717', '98840', '2868', '17727', '61847', '32417', '91009', '12117', '35077', '36279', '87552', '12441', '70576', '505', '57332', '56191', '33846', '78645', '58506', '48692', '27712', '86722', '46677', '81512', '14807', '30337', '7863', '457', '71893', '32256', '21897', '95811', '73552', '24929', '4893', '49448', '33836', '24781', '43236', '30853', '81985', '60625', '20087', '30674', '39280', '10779', '97785', '11298', '65057', '17589', '44085', '10126', '72351', '4165', '30539', '97124', '20545', '50087', '13490', '4055', '62365', '24046', '63110', '99045', '47122', '74389', '29843', '89082', '71686', '40321', '15789', '63062', '23403', '27718', '72550', '36098', '47309', '60568', '22266', '20542', '59018', '76386', '89381', '9164', '42093', '19931', '90426', '73695', '26948', '17363', '15727', '75439', '45425', '42518', '32966', '80703', '4659', '74566', '22464', '49383', '53167', '80720', '45464', '179', '29789', '19444', '97663', '6466', '51290', '56374', '17869', '97520', '45148', '92149', '54757', '94455', '80125', '98584', '70174', '77708', '59260', '50275', '49211', '60730', '29875', '90935', '29676', '95390', '18263', '73189', '41364', '39707', '75110', '59156', '11770', '38655', '55173', '62547', '16328', '70585', '81958', '61191', '90906', '45978', '59134', '24609', '32956', '30518', '27341', '84512', '24987', '65295', '75210', '97931', '56717', '6751', '83338', '82808', '96570', '27038', '93274', '399', '9330', '54319', '24848', '39269', '92692', '50625', '52306', '93262', '48756', '66694', '50517', '41763', '42231', '8589', '41923', '87839', '9579', '5104', '41634', '63030', '15100', '59593', '38288', '88852', '81957', '84267', '56380', '89329', '74372', '16026', '15795', '29456', '93173', '76367', '37598', '68153', '72405', '55181', '77543', '30730', '27564', '98764', '48711', '22180', '51352', '80150', '50091', '19481', '97962', '74879', '75369', '84410', '54712', '68399', '60910', '57922', '8896', '94118', '5276', '54049', '11614', '98854', '65979', '75156', '39456', '30226', '27357', '71087', '30814', '6416', '56480', '70259', '56525', '69945', '63874', '87262', '97398', '77972', '76745', '97191', '26739', '10957', '48743', '18175', '74688', '49853', '83675', '79232', '72211', '77255', '44445', '81053', '5506', '21366', '99015', '57598', '9467', '65131', '73794', '25679', '3609', '70041', '14653', '93792', '7076', '87326', '12978', '20886', '24089', '82187', '55203', '96649', '51205', '26250', '98219', '64275', '18634', '94004', '50999', '99611', '72380', '28685', '33246', '49116', '44972', '44273', '26558', '84799', '51310', '39480', '16713', '34371', '94308', '38743', '14592', '47430', '74505', '38866', '69596', '84705', '20236', '49353', '28259', '16255', '27665', '94271', '98652', '18279', '4805', '57471', '7944', '72567', '55603', '47327', '32268', '1256', '47608', '80225', '15744', '75492', '74652', '96812', '77976', '72595', '86791', '97736', '83149', '57030', '58553', '7121', '53544', '94894', '57236', '55850', '52762', '36303', '97740', '25314', '43196', '74248', '52320', '60448', '10606', '7010', '21765', '20357', '61825', '34331', '15898', '67485', '70353', '41657', '49212', '21628', '11783', '90514', '33883', '85668', '73432', '55374', '53880', '864', '66975', '51190', '18268', '45344', '99680', '91098', '3893', '90773', '55611', '5914', '14977', '29463', '38339', '94390', '15379', '74717', '78721', '10689', '90966', '33191', '32384', '4363', '61992', '37456', '28402', '51633', '14358', '99226', '23024', '22114', '86438', '70682', '27242', '46237', '92467', '83997', '71994', '59256', '44728', '51372', '50328', '66925', '16487', '7348', '69501', '23328', '55945', '98064', '92561', '5275', '67097', '37616', '8561', '14378', '83858', '44306', '60514', '89238', '28781', '11864', '31540', '3809', '16629', '4336', '70961', '10787', '58719', '39587', '46840', '48993', '18578', '30890', '71438', '29672', '54785', '77570', '25012', '70696', '25935', '279', '74327', '33944', '55060', '72194', '28269', '47600', '2132', '56618', '77029', '26410', '25682', '13744', '31968', '27115', '57065', '49313', '31237', '70475', '38549', '33723', '66622', '19646', '93518', '4052', '29390', '10163', '19879', '10765', '71024', '71339', '69114', '78385', '64440', '79577', '48109', '81114', '4708', '44863', '72410', '71161', '55491', '3433', '88675', '48496', '97875', '17184', '840', '16960', '32880', '48533', '82430', '44005', '36654', '96011', '16158', '21530', '88485', '91231', '77831', '39266', '9833', '96154', '89200', '2021', '57813', '897', '27643', '65750', '27447', '29043', '50691', '74683', '78027', '99590', '6052', '22867', '4567', '3918', '27143', '77750', '32003', '14011', '59006', '33140', '12165', '72413', '89931', '59389', '45281', '52596', '31993', '88613', '85332', '8016', '34419', '80901', '76317', '34932', '71099', '23354', '19296', '42671', '12646', '47189', '67982', '79803', '28449', '6229', '44873', '48970', '10012', '6871', '88240', '1277', '71039', '47959', '45051', '74358', '68394', '49093', '78063', '26981', '8704', '18470', '14458', '38195', '75691', '64683', '44910', '93423', '67591', '36833', '63204', '29507', '38894', '59524', '12141', '36543', '86914', '91250', '45459', '30156', '119', '95390', '75872', '28827', '67609', '99869', '91237', '70358', '60292', '3499', '3133', '45478', '5800', '47576', '50558', '71500', '65579', '84252', '68999', '59495', '79395', '28634', '44121', '26503', '11922', '22058', '34164', '32668', '10422', '62793', '41303', '2030', '76875', '17151', '42101', '22133', '82998', '19499', '14247', '95259', '746', '38449', '72450', '99138', '61259', '65623', '13439', '45597', '55234', '19347', '70266', '4828', '79316', '16983', '49723', '95468', '9747', '81126', '31999', '33796', '15270', '60644', '61048', '98848', '26685', '40980', '23733', '13676', '98363', '17970', '76451', '42657', '13165', '27870', '49193', '56279', '34647', '1652', '47674', '95555', '80556', '52419', '61110', '26124', '49097', '90600', '53605', '97544', '8694', '94739', '59652', '41807', '87155', '83539', '61468', '20614', '12652', '86562', '82894', '94360', '56182', '30513', '65379', '6522', '86486', + '37674', '44977', '54370', '97381', '60218', '2423', '99591', '69913', '26507', '19708', '6279', '58955', '20126', '1495', '57894', '7638', '38700', '77148', '36844', '7539', '91452', '6914', '74349', '66850', '49104', '6516', '58535', '20851', '27859', '32881', '72919', '28203', '32882', '2419', '77583', '63822', '37703', '66793', '65784', '62281', '55867', '70703', '89344', '1498', '33770', '87176', '95636', '64891', '90736', '95521', '10989', '5237', '99010', '21106', '11422', '1831', '67239', '52557', '36468', '71713', '39637', '49574', '50455', '14953', '96900', '70852', '96982', '4341', '44585', '95651', '79669', '29652', '87294', '74692', '16221', '768', '35380', '21352', '50907', '27259', '11718', '5017', '55964', '94137', '52347', '10595', '12968', '85602', '97965', '18836', '90511', '70960', '97336', '44575', '23791', '42195', '64776', '29363', '42379', '1805', '28919', '6772', '78143', '54797', '27362', '56149', '59048', '38567', '6339', '27787', '42167', '45990', '95532', '54839', '26572', '38496', '89797', '6634', '16468', '24898', '66814', '98126', '31762', '36133', '64539', '43167', '87022', '61295', '30364', '89249', '25756', '63570', '91484', '10564', '79648', '5756', '41376', '61897', '40388', '88927', '62891', '79708', '25495', '22204', '33892', '36871', '19879', '58646', '57061', '73100', '75831', '20029', '67462', '54675', '7766', '2409', '24506', '7877', '11720', '86252', '9897', '8080', '70684', '74497', '2242', '24604', '31969', '83999', '56635', '5283', '64971', '79152', '27470', '89042', '22835', '21476', '50292', '56081', '96342', '32763', '84487', '64856', '79152', '64656', '72169', '69971', '93094', '52804', '80917', '53152', '56016', '28496', '79110', '17133', '12581', '91742', '78929', '2676', '46700', '59528', '93808', '4535', '54035', '40161', '62796', '3598', '97088', '13599', '36337', '73395', '17494', '86275', '62058', '61937', '87747', '94883', '90677', '88544', '72553', '50210', '75481', '64378', '74464', '21659', '30970', '71989', '84846', '72289', '88716', '39143', '8487', '4912', '91013', '18623', '19122', '36507', '76438', '7516', '67970', '72350', '69873', '33635', '55983', '69008', '49545', '3134', '60056', '52509', '63304', '15560', '23651', '81090', '7027', '8317', '33060', '37295', '51961', '53037', '97431', '40512', '23536', '25168', '78455', '85613', '12304', '40733', '99890', '51238', '55439', '96201', '73559', '92533', '90173', '16721', '6078', '29854', '38894', '31117', '63040', '86795', '81786', '21149', '38998', '61811', '48622', '73019', '59296', '13576', '92559', '36300', '77294', '26794', '50912', '98380', '13176', '57746', '75286', '15330', '40921', '7337', '4664', '20384', '4674', '44516', '27633', '31950', '88210', '54536', '9839', '80137', '77491', '18434', '45152', '96942', '41005', '76103', '34825', '86869', '14772', '13384', '21051', '37348', '34434', '97210', '54960', '26598', '60981', '41889', '6446', '64492', '95310', '86236', '81885', '35684', '16539', '98476', '32028', '96470', '6318', '99576', '93935', '48609', '86090', '2476', '65576', '80636', '44817', '99646', '98963', '20486', '26261', '27334', '72946', '82023', '33506', '80193', '13762', '98133', '21134', '33268', '63477', '74609', '30454', '51477', '93391', '96805', '68653', '2714', '63642', '51520', '22972', '13305', '96058', '42336', '74461', '31597', '12050', '81712', '37977', '25718', '4834', '56608', '75731', '406', '28585', '63924', '23702', '29849', '16941', '91921', '65842', '76525', '68534', '50902', '17609', '23852', '53703', '31286', '58526', '9633', '87596', '10654', '2085', '52766', '22135', '76524', '32295', '90072', '70078', '77786', '93741', '87320', '70309', '44024', '95286', '12361', '29682', '59766', '26685', '90686', '81691', '49704', '23431', '53955', '39023', '47261', '1530', '58265', '80065', '95620', '90621', '63760', '90676', '81653', '36397', '20252', '81754', '20256', '67098', '7838', '49408', '88400', '87941', '84533', '6570', '22567', '18850', '55472', '40129', '48425', '23497', '39308', '34698', '53092', '89480', '47785', '57282', '25508', '19006', '50604', '86917', '9436', '88921', '3168', '70537', '3185', '34988', '5462', '69482', '45768', '91955', '56898', '15307', '99731', '89292', '19356', '20646', '66712', '7281', '12856', '31174', '19577', '8726', '62971', '33008', '37118', '59055', '84101', '68445', '91957', '47526', '15627', '79914', '20013', '26147', '80821', '56372', '74205', '28531', '25352', '51775', '93948', '55212', '17863', '91521', '74911', '88160', '2360', '98260', '18294', '62402', '84268', '9580', '42668', '1467', '40059', '5221', '4216', '9917', '35420', '16496', '34369', '50253', '95234', '95114', '84193', '28322', '37031', '81284', '88628', '36782', '42572', '73347', '66188', '43342', '77285', '16513', '89064', '63066', '72645', '67075', '48208', '18181', '77898', '65795', '53707', '39856', '92883', '92567', '49733', '30236', '10273', '53029', '69773', '78379', '72108', '47696', '97557', '95184', '14688', '29853', '62694', '70431', '88435', '58799', '21883', '99866', '69178', '55870', '14414', '85274', '27321', '55555', '613', '15067', '88217', '73655', '99548', '13631', '78789', '36690', '7952', '60830', '77438', '40059', '95602', '43097', '3429', '93731', '90537', '2932', '35702', '16125', '6652', '39632', '39349', '9910', '38103', '78608', '73565', '48556', '28978', '7128', '82326', '53980', '28059', '28212', '87101', '77752', '99170', '56753', '30484', '71470', '32607', '24674', '32687', '25098', '94712', '64024', '48239', '90408', '17316', '99243', '3656', '67402', '48009', '98427', '52800', '56024', '4417', '89747', '93338', '18758', '56411', '44810', '82456', '30808', '75470', '67115', '66876', '53906', '78403', '56059', '34383', '60056', '89136', '7237', '11129', '21351', '78662', '43606', '37454', '45465', '9292', '38099', '81699', '50195', '49368', '47503', '44605', '6523', '81478', '37910', '397', '20256', '6835', '2787', '80383', '4241', '65986', '83870', '21205', '10879', '26593', '44357', '72604', '56131', '43423', '80206', '26240', '87198', '99445', '53504', '10632', '2465', '31793', '89575', '64184', '39988', '60049', '87100', '37151', '61585', '82180', '52065', '72519', '72935', '3201', '5862', '20560', '95339', '21661', '17533', '17182', '71189', '91564', '57999', '35490', '94773', '95056', '51583', '59394', '10727', '8655', '48123', '10701', '25314', '20100', '6533', '46435', '43188', '23001', '23018', '76637', '32018', '36603', '18701', '9550', '61550', '47541', '36500', '67507', '81574', '95490', '69169', '32584', '30045', '64699', '83539', '89396', '42517', '61979', '41528', '8271', '88377', '61423', '1158', '89724', '70789', '14886', '64823', '56675', '97747', '23990', '58495', '82064', '17062', '90258', '86854', '93304', '12925', '49975', '45074', '87155', '72223', '67344', '42733', '42516', '40110', '15444', '88285', '39371', '23198', '61544', '90205', '6192', '15718', '19803', '92712', '20081', '31397', '5555', '70463', '19521', '80401', '74097', '32060', '26495', '20507', '40473', '1449', '57215', '46142', '39303', '50359', '35898', '46908', '90752', '7823', '27416', '73770', '98790', '17907', '29999', '76417', '49926', '76752', '21608', '26524', '88209', '6000', '88897', '19541', '41451', '59538', '56560', '1456', '67828', '82407', '45722', '93344', '54279', '78594', '38354', '93807', '10929', '91560', '60681', '70615', '32527', '10108', '48303', '63134', '28500', '18257', '57081', '24801', '99077', '52197', '15390', '52300', '57116', '417', '7503', '20054', '75315', '81359', '69091', '18853', '2465', '25600', '13522', '74575', '12661', '83071', '15191', '27543', '21730', '60853', '18961', '14773', '89185', '33694', '51143', '1449', '68831', '78062', '65173', '32697', '41674', '9429', '22156', '96022', '46305', '97534', '5685', '48870', '89988', '20686', '66705', '6865', '94250', '16872', '13178', '7420', '73531', '92723', '60620', '48843', '74207', '60016', '50943', '62699', '63507', '76537', '87066', '76922', '24711', '34809', '5021', '31293', '53854', '77607', '52322', '10934', '50284', '87804', '36730', '86946', '80749', '43325', '97958', '7362', '39582', '10042', '42053', '66236', '69931', '23463', '87996', '33563', '4468', '32905', '50815', '79478', '28658', '46018', '23186', '26080', '13494', '6237', '42762', '86440', '77407', '10426', '62902', '73251', '36861', '92357', '98754', '1839', '46391', '11420', '27132', '93028', '39609', '42015', '68218', '54228', '5456', '38705', '64307', '49483', '878', '54360', '54480', '66684', '55089', '4537', '82073', '72602', '96238', '56708', '58625', '32991', '74205', '72868', '79086', '64250', '56376', '10621', '76607', '47706', '72760', '70303', '60715', '14644', '44186', '36264', '29489', '14184', '62699', '30567', '16700', '31222', '15650', '1500', '22950', '54628', '41004', '96094', '70028', '74178', '65328', '26605', '63076', '75271', '79285', '8151', '42101', '56362', '25961', '87864', '972', '29510', '2747', '8877', '9780', '61052', '84105', '15573', '27475', '44570', '25334', '18517', '44237', '84094', '67524', '76761', '65678', '79284', '2462', '42631', '22696', '19223', '29728', '67742', '11883', '59027', '12377', '80538', '2165', '17377', '15030', '49838', '23920', '26025', '68179', '75894', '43783', '97106', '75558', '35528', '52081', '16951', '68855', '402', '21459', '97550', '16948', '5369', '4641', '2663', '15233', '79974', '71093', '15234', '42690', '22322', '54282', '95845', '90010', '40530', '88298', '41885', '7079', '6098', '72786', '36603', '77378', '48393', '45723', '41996', '96025', '89297', '75586', '8422', '24360', '170', '46036', '46725', '67944', '74029', '73069', '45371', '99916', '71085', '42608', '89904', '6393', '51274', '42729', '58924', '82497', '64143', '88622', '18818', '89041', '56090', '21369', '78224', '90450', '45488', '58830', '4133', '98062', '81113', '11285', '51457', '3183', '38800', '65278', '42169', '28602', '52648', '44683', '75647', '11778', '32151', '33528', '23773', '68268', '23367', '70964', '23548', + '35575', '67570', '77681', '74158', '25374', '62714', '43100', '4977', '51678', '83460', '29755', '15890', '64626', '54044', '14793', '64339', '94008', '97126', '49202', '33889', '12601', '12275', '56123', '94557', '68226', '67200', '9374', '70687', '29211', '8039', '14598', '74548', '37433', '98991', '29933', '37203', '23973', '96482', '64774', '58350', '61781', '31824', '57193', '26476', '21814', '32297', '32627', '44277', '33876', '55468', '81715', '82505', '61462', '20324', '84293', '40116', '51087', '43594', '6854', '59077', '39841', '26023', '22777', '66859', '82460', '89515', '41712', '33711', '71875', '10685', '12655', '50138', '31063', '37040', '95819', '38919', '27391', '29833', '34350', '65646', '7697', '2688', '41146', '13241', '50305', '86568', '24487', '78741', '96370', '21015', '31719', '39750', '25014', '72415', '8486', '90668', '51143', '49488', '21057', '92803', '53528', '39550', '76039', '44185', '32404', '30217', '19796', '38084', '49161', '80140', '20241', '39357', '68908', '93083', '77231', '6952', '36322', '50790', '623', '29730', '13616', '57546', '17434', '93811', '35148', '81419', '40250', '40329', '89126', '72402', '16053', '27107', '28919', '16829', '96582', '65057', '28416', '30801', '77742', '27420', '73118', '89352', '54706', '23035', '88413', '64608', '61930', '15037', '47327', '59596', '18700', '57576', '63628', '56823', '60091', '68209', '21001', '14962', '72257', '83802', '33721', '86343', '11133', '65737', '68477', '90725', '86869', '98403', '47393', '25356', '61372', '8873', '19888', '48836', '66005', '23531', '72520', '26461', '78508', '28213', '96394', '22983', '37856', '71814', '27425', '72753', '27511', '65471', '38592', '3683', '24652', '64505', '92543', '53201', '40639', '99542', '53425', '35321', '47669', '14134', '47727', '48202', '71931', '32119', '50086', '50266', '67159', '89317', '81905', '30315', '49154', '8690', '69365', '56881', '46473', '64100', '38365', '59377', '65630', '54871', '52745', '91536', '16106', '70066', '62063', '84530', '88103', '33599', '51063', '87299', '41880', '25335', '51252', '42788', '13568', '1721', '62424', '83308', '36787', '91536', '92555', '27600', '24030', '12267', '66336', '30242', '7183', '67624', '28471', '48593', '79766', '31178', '47818', '94522', '88855', '45262', '43670', '18065', '25062', '44558', '37189', '69225', '35216', '42683', '26289', '72816', '31947', '65871', '45715', '59452', '22014', '56669', '60331', '33450', '60601', '95047', '30789', '90107', '81565', '32266', '3252', '5446', '58756', '55370', '34034', '81071', '2560', '39054', '39564', '15010', '5389', '60002', '53320', '49545', '48444', '31415', '39278', '79879', '30148', '10186', '60358', '29011', '14419', '95159', '94815', '55251', '90910', '80582', '92304', '11697', '60061', '38577', '84439', '76196', '34542', '50963', '36294', '11123', '59763', '29873', '47383', '12979', '22119', '21723', '64725', '48377', '77132', '9817', '79920', '47653', '60069', '12924', '53808', '55962', '66969', '13757', '60615', '10994', '9138', '34119', '58436', '64407', '75170', '73524', '51864', '94183', '86847', '15585', '57616', '96267', '5340', '52929', '49096', '50291', '5559', '32382', '84077', '6598', '87921', '59719', '31726', '44772', '63373', '75420', '66829', '47275', '98264', '61387', '94945', '44540', '50098', '13078', '44729', '95332', '63555', '30782', '63203', '15071', '60996', '72812', '17418', '80215', '37610', '30670', '44674', '74822', '15471', '25236', '16266', '76213', '35820', '19567', '8715', '72003', '90606', '1434', '53545', '88170', '75014', '62287', '35436', '38669', '12927', '83877', '38622', '28313', '82884', '73969', '38671', '10450', '24158', '22941', '73162', '86548', '42482', '95315', '92016', '96156', '44012', '35962', '6366', '3881', '74300', '26248', '30182', '19164', '67105', '66771', '52587', '69894', '61820', '16551', '50743', '10096', '69030', '24451', '89165', '23929', '96291', '30685', '64413', '19913', '9049', '71383', '61684', '45384', '45927', '81840', '49521', '89594', '30055', '83430', '14930', '60316', '86585', '99375', '80170', '14207', '19584', '20067', '82874', '30159', '46647', '6942', '66777', '32638', '55662', '75470', '77622', '26893', '96149', '14373', '33252', '50574', '7945', '20696', '56662', '94348', '3384', '20956', '89668', '99052', '65131', '56847', '17589', '16419', '2670', '10705', '59587', '92902', '92424', '48570', '11034', '69149', '35733', '17315', '84966', '69353', '69590', '52834', '32561', '6049', '50156', '71676', '76423', '32361', '61509', '8845', '75709', '35956', '21912', '31188', '59083', '43459', '38614', '92206', '55645', '38737', '34193', '6451', '94163', '24326', '49976', '71600', '58024', '67160', '4365', '38270', '59558', '80834', '60739', '54318', '19738', '42196', '43191', '13463', '88914', '99239', '66869', '75691', '33085', '4323', '7170', '46184', '41423', '89835', '46877', '20349', '14365', '32727', '35322', '841', '23597', '43370', '57527', '73250', '32553', '71489', '44617', '98323', '37672', '59549', '96023', '63176', '13524', '15621', '30448', '28136', '45549', '3513', '64153', '19839', '24219', '41987', '51083', '90268', '52052', '31430', '4727', '99409', '43595', '82374', '61251', '51470', '66562', '98724', '23529', '53895', '67562', '87573', '89964', '30821', '15733', '33062', '86963', '33450', '75338', '32570', '14453', '38080', '36335', '84226', '52790', '42883', '61156', '42789', '57846', '60096', '29946', '80178', '15882', '1971', '60722', '62458', '8754', '59991', '89321', '584', '70565', '36458', '21226', '23561', '9837', '39364', '23065', '30675', '9306', '40085', '52082', '89976', '73283', '77851', '36174', '54470', '63250', '72111', '70853', '26723', '42590', '91230', '47512', '13983', '70898', '70927', '40721', '30642', '41628', '90010', '27306', '1933', '43304', '44499', '87890', '22201', '89249', '63935', '48438', '58588', '1061', '70061', '63075', '9676', '65820', '82156', '82668', '111', '54350', '10328', '23466', '98936', '18285', '53919', '32422', '84859', '58387', '24022', '32423', '6010', '56417', '49452', '69999', '14885', '47102', '59577', '24999', '75984', '96464', '59088', '85987', '71442', '88789', '4753', '8229', '76883', '15284', '90610', '40507', '78882', '55575', '25315', '7214', '70602', '4796', '35767', '54657', '42153', '16050', '93607', '99249', '77236', '59949', '52871', '47837', '33534', '30023', '89137', '99938', '35824', '50775', '30282', '82798', '53312', '65277', '68375', '91445', '58166', '43344', '6589', '82515', '34632', '78588', '152', '67554', '15877', '74334', '32783', '45147', '39483', '92067', '59029', '38298', '55229', '28268', '85140', '33451', '15424', '46695', '23201', '83329', '28372', '19518', '89198', '33305', '43892', '470', '37662', '9407', '14376', '80310', '21459', '72381', '80414', '88305', '69073', '63101', '91054', '47190', '48595', '24696', '41426', '35133', '94399', '21790', '55040', '73279', '20809', '67805', '94115', '58633', '78053', '89444', '4112', '8', '34517', '22106', '85934', '86814', '53333', '93437', '85062', '32791', '72744', '99843', '51161', '22730', '34908', '82918', '92566', '22467', '41226', '98518', '29235', '94042', '84371', '79100', '25214', '7764', '59427', '47891', '61092', '23775', '13641', '30837', '77377', '43032', '38441', '29462', '20300', '19070', '20982', '73987', '87836', '68062', '6419', '51563', '40084', '85694', '86677', '47142', '27222', '17844', '19158', '45120', '88524', '74724', '73229', '42470', '38751', '1132', '28603', '61188', '55021', '88825', '58005', '62411', '8843', '94852', '93664', '39253', '27473', '247', '43824', '1804', '8905', '11509', '95659', '7811', '80691', '15779', '49794', '8991', '76099', '29223', '36060', '85399', '41369', '22885', '38473', '22376', '50446', '89578', '25818', '61333', '78787', '47605', '83654', '99068', '52120', '48367', '86381', '19803', '72600', '31998', '37755', '88031', '83969', '42319', '27974', '35780', '93662', '46808', '60529', '15491', '10447', '48829', '33886', '68333', '44855', '86554', '64794', '66376', '58222', '14021', '52043', '56375', '1300', '38105', '89159', '97456', '26800', '93124', '3673', '32279', '30658', '84475', '3708', '93952', '39245', '91980', '55333', '79440', '64407', '46559', '60759', '10688', '49872', '45810', '87405', '66932', '56530', '57751', '9619', '27361', '6356', '65848', '7524', '20273', '22362', '20504', '28042', '39475', '51677', '85733', '32426', '54558', '17222', '56485', '34928', '90917', '70528', '51732', '61014', '98420', '67265', '41383', '3883', '47642', '53324', '93679', '93088', '57534', '44449', '46779', '81482', '54279', '80135', '11216', '92545', '18426', '96005', '57801', '21898', '5104', '83467', '72015', '43783', '89674', '57468', '96686', '95167', '38507', '95187', '64923', '71214', '42834', '93219', '47342', '24476', '84834', '29080', '86533', '30687', '68400', '26933', '37396', '65169', '89767', '20642', '53843', '85167', '77306', '46723', '68501', '4243', '35044', '15950', '40388', '53630', '76125', '10816', '83285', '4120', '11402', '91344', '95169') + DROP TABLE IF EXISTS test_in From a9b94b11d6513641ee8a04ca65b4298bdcb711d4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Mar 2020 13:29:49 +0300 Subject: [PATCH 059/147] empty commit From 74327e7f4776d38f3d3498ec4471bac4569d49c5 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 20 Mar 2020 16:18:26 +0400 Subject: [PATCH 060/147] remove settings from query --- dbms/programs/copier/ClusterCopier.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index c5266b3e43a..5f4480e6d5c 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -603,10 +603,16 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t original_table.second + "_piece_" + toString(current_piece_number)); + Settings settings_push = task_cluster->settings_push; + + /// It is important, ALTER ATTACH PARTITION must be done synchronously + /// And we will execute this ALTER query on each replica of a shard. + /// It is correct, because this query is idempotent. + settings_push.replication_alter_partitions_sync = 2; + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + " ATTACH PARTITION " + partition_name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); @@ -616,7 +622,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t task_table.cluster_push, query_alter_ast_string, nullptr, - &task_cluster->settings_push, + &settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); From 66ccbf5d11f8814a937e7b12ba18661cc0fa1ec5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Mar 2020 20:43:01 +0300 Subject: [PATCH 061/147] Pass query to the IStorage::getQueryProcessingStage() Will be used to return correct stage with optimize_skip_unused_shards. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Storages/IStorage.h | 5 ++++- dbms/src/Storages/LiveView/StorageBlocks.h | 2 +- dbms/src/Storages/StorageBuffer.cpp | 4 ++-- dbms/src/Storages/StorageBuffer.h | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 4 ++-- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMerge.cpp | 8 ++++---- dbms/src/Storages/StorageMerge.h | 2 +- 11 files changed, 19 insertions(+), 16 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 4fe83afa48d..779bbbadb85 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -508,7 +508,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl(bool try_move_to_prewhere) } if (storage && !options.only_analyze) - from_stage = storage->getQueryProcessingStage(*context); + from_stage = storage->getQueryProcessingStage(*context, query_ptr); /// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing. bool first_stage = from_stage < QueryProcessingStage::WithMergeableState diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index e2a1afd95f8..cc82a0a9ce9 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -220,8 +220,11 @@ public: /** Returns stage to which query is going to be processed in read() function. * (Normally, the function only reads the columns from the list, but in other cases, * for example, the request can be partially processed on a remote server.) + * + * SelectQueryInfo is required since the stage can depends on the query + * (see Distributed() engine and optimize_skip_unused_shards). */ - virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &) const { return QueryProcessingStage::FetchColumns; } + virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const { return QueryProcessingStage::FetchColumns; } /** Watch live changes to the table. * Accepts a list of columns to read, as well as a description of the query, diff --git a/dbms/src/Storages/LiveView/StorageBlocks.h b/dbms/src/Storages/LiveView/StorageBlocks.h index e2c3a8b10f4..5d06172ffb8 100644 --- a/dbms/src/Storages/LiveView/StorageBlocks.h +++ b/dbms/src/Storages/LiveView/StorageBlocks.h @@ -26,7 +26,7 @@ public: return std::make_shared(table_id, columns, std::move(pipes), to_stage); } std::string getName() const override { return "Blocks"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const override { return to_stage; } Pipes read( const Names & /*column_names*/, diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 217d474defe..4a4c92129bf 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -135,7 +135,7 @@ private: }; -QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context) const +QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const { if (destination_id) { @@ -144,7 +144,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - return destination->getQueryProcessingStage(context); + return destination->getQueryProcessingStage(context, query_ptr); } return QueryProcessingStage::FetchColumns; diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index d4c6f3e6078..487fe6bcb9d 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -54,7 +54,7 @@ public: std::string getName() const override { return "Buffer"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; Pipes read( const Names & column_names, diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 77ea7bd97f2..aee2b597a44 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -369,7 +369,7 @@ static QueryProcessingStage::Enum getQueryProcessingStageImpl(const Context & co : QueryProcessingStage::WithMergeableState; } -QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context) const +QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, const ASTPtr &) const { auto cluster = getCluster(); return getQueryProcessingStageImpl(context, cluster); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index b312b55c766..74b987e29f9 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -66,7 +66,7 @@ public: bool isRemote() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; Pipes read( const Names & column_names, diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index e104dcd8edc..3fb25bf8275 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -171,9 +171,9 @@ StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const return result; } -QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context) const +QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const { - return getTargetTable()->getQueryProcessingStage(context); + return getTargetTable()->getQueryProcessingStage(context, query_ptr); } Pipes StorageMaterializedView::read( diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 6951f11038e..34704644b00 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -59,7 +59,7 @@ public: void checkTableCanBeDropped() const override; void checkPartitionCanBeDropped(const ASTPtr & partition) override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 9e274c56454..f3322c7dfff 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -136,7 +136,7 @@ bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, cons } -QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context) const +QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; @@ -150,7 +150,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & if (table.get() != this) { ++selected_table_size; - stage_in_source_tables = std::max(stage_in_source_tables, table->getQueryProcessingStage(context)); + stage_in_source_tables = std::max(stage_in_source_tables, table->getQueryProcessingStage(context, query_ptr)); } iterator->next(); @@ -287,7 +287,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer return pipes; } - if (processed_stage <= storage->getQueryProcessingStage(*modified_context)) + if (processed_stage <= storage->getQueryProcessingStage(*modified_context, query_info.query)) { /// If there are only virtual columns in query, you must request at least one other column. if (real_column_names.empty()) @@ -295,7 +295,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer pipes = storage->read(real_column_names, modified_query_info, *modified_context, processed_stage, max_block_size, UInt32(streams_num)); } - else if (processed_stage > storage->getQueryProcessingStage(*modified_context)) + else if (processed_stage > storage->getQueryProcessingStage(*modified_context, query_info.query)) { modified_query_info.query->as()->replaceDatabaseAndTable(source_database, table_name); diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 1b7bb2fabe8..5d7f527d8b6 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -31,7 +31,7 @@ public: NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context &) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const override; Pipes read( const Names & column_names, From 4707dd827a5c3a9aef0166f619b07ef85eea5819 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 24 Mar 2020 10:51:54 +0300 Subject: [PATCH 062/147] StorageDistributed: cleanup skipping unused shards --- dbms/src/Storages/StorageDistributed.cpp | 99 ++++++++++++------------ dbms/src/Storages/StorageDistributed.h | 8 +- 2 files changed, 56 insertions(+), 51 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index aee2b597a44..048d2091e2a 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -236,7 +236,7 @@ void replaceConstantExpressions(ASTPtr & node, const Context & context, const Na visitor.visit(node); } -} +} // \anonymous /// For destruction of std::unique_ptr of type that is incomplete in class definition. @@ -383,9 +383,7 @@ Pipes StorageDistributed::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { - auto cluster = getCluster(); - - const Settings & settings = context.getSettingsRef(); + auto cluster = getOptimizedCluster(context, query_info.query); const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); @@ -405,50 +403,8 @@ Pipes StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, StorageID{remote_database, remote_table}, scalars, has_virtual_shard_num_column, context.getExternalTables()); - UInt64 force = settings.force_optimize_skip_unused_shards; - if (settings.optimize_skip_unused_shards) - { - ClusterPtr smaller_cluster; - auto table_id = getStorageID(); - - if (has_sharding_key) - { - smaller_cluster = skipUnusedShards(cluster, query_info, context); - - if (smaller_cluster) - { - cluster = smaller_cluster; - LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " - "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " - " " << makeFormattedListOfShards(cluster)); - } - } - - if (!smaller_cluster) - { - LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << - (has_sharding_key ? "" : " (no sharding key)") << ": " - "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - " - "the query will be sent to all shards of the cluster"); - - if (force) - { - std::stringstream exception_message; - if (!has_sharding_key) - exception_message << "No sharding key"; - else - exception_message << "Sharding key " << sharding_key_column_name << " is not used"; - - if (force == FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_ALWAYS) - throw Exception(exception_message.str(), ErrorCodes::UNABLE_TO_SKIP_UNUSED_SHARDS); - if (force == FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_HAS_SHARDING_KEY && has_sharding_key) - throw Exception(exception_message.str(), ErrorCodes::UNABLE_TO_SKIP_UNUSED_SHARDS); - } - } - } - return ClusterProxy::executeQuery( - select_stream_factory, cluster, modified_query_ast, context, settings, query_info); + select_stream_factory, cluster, modified_query_ast, context, context.getSettingsRef(), query_info); } @@ -631,6 +587,51 @@ ClusterPtr StorageDistributed::getCluster() const return owned_cluster ? owned_cluster : global_context.getCluster(cluster_name); } +ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, const ASTPtr & query_ptr) const +{ + ClusterPtr cluster = getCluster(); + const Settings & settings = context.getSettingsRef(); + auto table_id = getStorageID(); + + if (!settings.optimize_skip_unused_shards) + return cluster; + + if (has_sharding_key) + { + ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, context); + + if (optimized) + { + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": " + "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " + " " << makeFormattedListOfShards(cluster)); + return optimized; + } + } + + LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << + (has_sharding_key ? "" : " (no sharding key)") << ": " + "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - " + "the query will be sent to all shards of the cluster"); + + UInt64 force = settings.force_optimize_skip_unused_shards; + if (force) + { + std::stringstream exception_message; + if (!has_sharding_key) + exception_message << "No sharding key"; + else + exception_message << "Sharding key " << sharding_key_column_name << " is not used"; + + if (force == FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_ALWAYS) + throw Exception(exception_message.str(), ErrorCodes::UNABLE_TO_SKIP_UNUSED_SHARDS); + if (force == FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_HAS_SHARDING_KEY && has_sharding_key) + throw Exception(exception_message.str(), ErrorCodes::UNABLE_TO_SKIP_UNUSED_SHARDS); + } + + return cluster; +} + void StorageDistributed::ClusterNodeData::flushAllData() { directory_monitor->flushAllData(); @@ -643,9 +644,9 @@ void StorageDistributed::ClusterNodeData::shutdownAndDropAllData() /// Returns a new cluster with fewer shards if constant folding for `sharding_key_expr` is possible /// using constraints from "PREWHERE" and "WHERE" conditions, otherwise returns `nullptr` -ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info, const Context & context) +ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const Context & context) const { - const auto & select = query_info.query->as(); + const auto & select = query_ptr->as(); if (!select.prewhere() && !select.where()) { diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 74b987e29f9..8eea29ba8d6 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -114,6 +114,12 @@ public: ClusterPtr getCluster() const; + /// Apply the following settings: + /// - optimize_skip_unused_shards + /// - force_optimize_skip_unused_shards + ClusterPtr getOptimizedCluster(const Context &, const ASTPtr & query_ptr) const; + ClusterPtr skipUnusedShards(ClusterPtr cluster, const ASTPtr & query_ptr, const Context & context) const; + ActionLock getActionLock(StorageActionBlockType type) override; String remote_database; @@ -164,8 +170,6 @@ protected: const String & relative_data_path_, bool attach); - ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info, const Context & context); - void createStorage(); String storage_policy; From 0e925ff7ad6370b64f449097919bd8c083e43cb5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Mar 2020 20:47:01 +0300 Subject: [PATCH 063/147] Cover DISTINCT for Distributed with one shard (test from #9046) v2: cover case when getQueryProcessingStage() returns Complete when it should not (pointed by @alexey-milovidov in https://github.com/ClickHouse/ClickHouse/pull/9808#issuecomment-602291996) --- ...mize_skip_unused_shards_DISTINCT.reference | 8 ++++++++ ...3_optimize_skip_unused_shards_DISTINCT.sql | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference create mode 100644 dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql diff --git a/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference b/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference new file mode 100644 index 00000000000..4ade9cd9c5d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.reference @@ -0,0 +1,8 @@ +distributed_group_by_no_merge +1 +1 +optimize_skip_unused_shards +1 +optimize_skip_unused_shards lack of WHERE +0 +1 diff --git a/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql b/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql new file mode 100644 index 00000000000..5b45bea9046 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01213_optimize_skip_unused_shards_DISTINCT.sql @@ -0,0 +1,19 @@ +CREATE TABLE IF NOT EXISTS local_01213 (id Int) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE IF NOT EXISTS dist_01213 AS local_01213 ENGINE = Distributed(test_cluster_two_shards_localhost, currentDatabase(), local_01213, id); + +-- at least two parts +INSERT INTO local_01213 SELECT toString(number) FROM numbers(2); +INSERT INTO local_01213 SELECT toString(number) FROM numbers(2); + +-- check that without merge we will have two rows +SELECT 'distributed_group_by_no_merge'; +SELECT DISTINCT id FROM dist_01213 WHERE id = 1 SETTINGS distributed_group_by_no_merge=1; +-- check that with merge there will be only one +SELECT 'optimize_skip_unused_shards'; +SELECT DISTINCT id FROM dist_01213 WHERE id = 1 SETTINGS optimize_skip_unused_shards=1; +-- check that querying all shards is ok +SELECT 'optimize_skip_unused_shards lack of WHERE'; +SELECT DISTINCT id FROM dist_01213 SETTINGS optimize_skip_unused_shards=1; + +DROP TABLE local_01213; +DROP TABLE dist_01213; From b8614de64af3e65c83ee35588082df373959a737 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 24 Mar 2020 10:53:42 +0300 Subject: [PATCH 064/147] Fix DISTINCT for Distributed and optimize_skip_unused_shards In case of optimize_skip_unused_shards the getQueryProcessingStage() can return QueryProcessingStage::WithMergeableState because the whole cluster has multiple nodes, but optimize_skip_unused_shards can trim it to 1 and in this case we need final merge that won't be done with QueryProcessingStage::WithMergeableState only with QueryProcessingStage::Complete. --- dbms/src/Storages/StorageDistributed.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 048d2091e2a..afc730fbf9d 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -369,9 +369,9 @@ static QueryProcessingStage::Enum getQueryProcessingStageImpl(const Context & co : QueryProcessingStage::WithMergeableState; } -QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, const ASTPtr &) const +QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const { - auto cluster = getCluster(); + auto cluster = getOptimizedCluster(context, query_ptr); return getQueryProcessingStageImpl(context, cluster); } From c1291e93483a3d5a6174c3a00e00225613e5b746 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 24 Mar 2020 15:55:35 +0300 Subject: [PATCH 065/147] tuple as literal --- .../evaluateConstantExpression.cpp | 69 ++++++++++++------- dbms/src/Parsers/ExpressionElementParsers.cpp | 14 ++-- dbms/src/Parsers/ExpressionElementParsers.h | 1 + .../Processors/Executors/PipelineExecutor.h | 1 + .../Impl/ConstantExpressionTemplate.cpp | 47 ++++++++++--- 5 files changed, 90 insertions(+), 42 deletions(-) diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 37e990fd5fc..fd5e2475df1 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -101,9 +101,9 @@ namespace using Conjunction = ColumnsWithTypeAndName; using Disjunction = std::vector; - Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr) + Disjunction analyzeEquals(const ASTIdentifier * identifier, const Field & value, const ExpressionActionsPtr & expr) { - if (!identifier || !literal) + if (!identifier || value.isNull()) { return {}; } @@ -117,7 +117,7 @@ namespace { ColumnWithTypeAndName column; // FIXME: what to do if field is not convertable? - column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); + column.column = type->createColumnConst(1, convertFieldToType(value, *type)); column.name = name; column.type = type; return {{std::move(column)}}; @@ -127,6 +127,16 @@ namespace return {}; } + Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr) + { + if (!identifier || !literal) + { + return {}; + } + + return analyzeEquals(identifier, literal->value, expr); + } + Disjunction andDNF(const Disjunction & left, const Disjunction & right) { if (left.empty()) @@ -172,33 +182,44 @@ namespace const auto * left = fn->arguments->children.front().get(); const auto * right = fn->arguments->children.back().get(); const auto * identifier = left->as(); - const auto * inner_fn = right->as(); - - if (!inner_fn) - { - return {}; - } - - const auto * tuple = inner_fn->children.front()->as(); - - if (!tuple) - { - return {}; - } Disjunction result; - for (const auto & child : tuple->children) + if (const auto * tuple_func = right->as(); tuple_func && tuple_func->name == "tuple") { - const auto * literal = child->as(); - const auto dnf = analyzeEquals(identifier, literal, expr); - - if (dnf.empty()) + const auto * tuple_elements = tuple_func->children.front()->as(); + for (const auto & child : tuple_elements->children) { - return {}; - } + const auto * literal = child->as(); + const auto dnf = analyzeEquals(identifier, literal, expr); - result.insert(result.end(), dnf.begin(), dnf.end()); + if (dnf.empty()) + { + return {}; + } + + result.insert(result.end(), dnf.begin(), dnf.end()); + } + } + else if (const auto * tuple_literal = right->as(); + tuple_literal && tuple_literal->value.getType() == Field::Types::Tuple) + { + const auto & tuple = tuple_literal->value.get(); + for (const auto & child : tuple) + { + const auto dnf = analyzeEquals(identifier, child, expr); + + if (dnf.empty()) + { + return {}; + } + + result.insert(result.end(), dnf.begin(), dnf.end()); + } + } + else + { + return {}; } return result; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index af676d94f77..59c63f89189 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -999,11 +999,9 @@ bool ParserCollectionOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Pos literal_begin = pos; Collection arr; - ParserLiteral literal_p; ++pos; - while (pos.isValid()) { if (!arr.empty()) @@ -1012,12 +1010,11 @@ bool ParserCollectionOfLiterals::parseImpl(Pos & pos, ASTPtr & node, { std::shared_ptr literal; - /// Parse one-element tuples (e.g. (1)) as single values for backward compatibility. + /// Parse one-element tuples (e.g. (1)) later as single values for backward compatibility. if (std::is_same_v && arr.size() == 1) - literal = std::make_shared(arr[0]); - else - literal = std::make_shared(arr); + return false; + literal = std::make_shared(arr); literal->begin = literal_begin; literal->end = ++pos; node = literal; @@ -1029,9 +1026,8 @@ bool ParserCollectionOfLiterals::parseImpl(Pos & pos, ASTPtr & node, } else { - std::stringstream msg; - msg << "comma or " << getTokenName(closing_bracket); - expected.add(pos, msg.str().c_str()); + String message = String("comma or ") + getTokenName(closing_bracket); + expected.add(pos, message.c_str()); return false; } } diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index b2923118225..b9d8d5db42c 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -239,6 +239,7 @@ private: TokenType closing_bracket; }; +/// A tuple of literals with same type. class ParserTupleOfLiterals : public IParserBase { public: diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index 673151bd5eb..20569d974ee 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -273,6 +273,7 @@ private: void executeSingleThread(size_t thread_num, size_t num_threads); void finish(); +public: String dumpPipeline() const; }; diff --git a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 83e449ee368..07506f7ca8e 100644 --- a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -148,11 +149,18 @@ private: info.type = std::make_shared(); else if (field_type == Field::Types::String) info.type = std::make_shared(); - else if (field_type == Field::Types::Array) + else if (field_type == Field::Types::Array || field_type == Field::Types::Tuple) { info.special_parser.is_array = true; info.type = applyVisitor(FieldToDataType(), info.literal->value); - auto nested_type = assert_cast(*info.type).getNestedType(); + + DataTypePtr nested_type; + if (auto array_type = typeid_cast(info.type.get())) + nested_type = array_type->getNestedType(); + else if (auto tuple_type = typeid_cast(info.type.get())) + nested_type = tuple_type->getElements()[0]; + else + throw Exception("Unexpected type " + info.type->getName(), ErrorCodes::LOGICAL_ERROR); /// It can be Array(Nullable(nested_type)) bool array_of_nullable = false; @@ -192,7 +200,18 @@ private: info.special_parser.is_nullable = true; } - info.type = std::make_shared(nested_type); + if (field_type == Field::Types::Tuple) + { + const auto & tuple = info.literal->value.get(); + DataTypes elements(tuple.size()); + for (size_t i = 0; i < tuple.size(); ++i) + elements[i] = nested_type; + info.type = std::make_shared(elements); + } + else + { + info.type = std::make_shared(nested_type); + } } else throw Exception(String("Unexpected literal type ") + info.literal->value.getTypeName() + ". It's a bug", @@ -408,18 +427,28 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co { /// TODO faster way to check types without using Parsers ParserArrayOfLiterals parser_array; + ParserTupleOfLiterals parser_tuple; + Tokens tokens_number(istr.position(), istr.buffer().end()); IParser::Pos iterator(tokens_number, settings.max_parser_depth); Expected expected; ASTPtr ast; - - if (!parser_array.parse(iterator, ast, expected)) + if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected)) return false; istr.position() = const_cast(iterator->begin); - const Field & array = ast->as().value; - auto array_type = applyVisitor(FieldToDataType(), array); - auto nested_type = assert_cast(*array_type).getNestedType(); + const Field & collection = ast->as().value; + auto collection_type = applyVisitor(FieldToDataType(), collection); + + DataTypePtr nested_type; + if (auto array_type = typeid_cast(collection_type.get())) + nested_type = array_type->getNestedType(); + else if (auto tuple_type = typeid_cast(collection_type.get())) + nested_type = tuple_type->getElements()[0]; + + if (!nested_type) + return false; + if (type_info.is_nullable) if (auto nullable = dynamic_cast(nested_type.get())) nested_type = nullable->getNestedType(); @@ -429,7 +458,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co (nested_type_info.isNativeInt() && type_info.nested_type == Type::Int64) || (nested_type_info.isFloat64() && type_info.nested_type == Type::Float64)) { - Field array_same_types = convertFieldToType(array, *complex_type, nullptr); + Field array_same_types = convertFieldToType(collection, *complex_type, nullptr); columns[column_idx]->insert(array_same_types); return true; } From 580a7c9f412a044761ac731ebf9050fecfa1ded5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 24 Mar 2020 21:00:57 +0300 Subject: [PATCH 066/147] Use overload over default value for IStorage::getQueryProcessingStage() Since clang-tidy complains about it [1]: 2020-03-24 09:00:53 /build/obj-x86_64-linux-gnu/../dbms/src/Storages/StorageMerge.cpp:139:42: error: default arguments on virtual or override methods are prohibited [google-default-arguments,-warnings-as-errors] 2020-03-24 09:00:53 QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & context, const ASTPtr & query_ptr) const [1]: https://clickhouse-builds.s3.yandex.net/9808/b8614de64af3e65c83ee35588082df373959a737/build_log_637307151_1585043194.txt And indeed, this is a good idea! --- dbms/src/Storages/IStorage.h | 3 ++- dbms/src/Storages/LiveView/StorageBlocks.h | 2 +- dbms/src/Storages/StorageBuffer.h | 2 +- dbms/src/Storages/StorageDistributed.h | 2 +- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMerge.h | 2 +- 6 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index cc82a0a9ce9..5e6d6f1fa22 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -224,7 +224,8 @@ public: * SelectQueryInfo is required since the stage can depends on the query * (see Distributed() engine and optimize_skip_unused_shards). */ - virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const { return QueryProcessingStage::FetchColumns; } + QueryProcessingStage::Enum getQueryProcessingStage(const Context &context) const { return getQueryProcessingStage(context, {}); } + virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const { return QueryProcessingStage::FetchColumns; } /** Watch live changes to the table. * Accepts a list of columns to read, as well as a description of the query, diff --git a/dbms/src/Storages/LiveView/StorageBlocks.h b/dbms/src/Storages/LiveView/StorageBlocks.h index 5d06172ffb8..fd856e27718 100644 --- a/dbms/src/Storages/LiveView/StorageBlocks.h +++ b/dbms/src/Storages/LiveView/StorageBlocks.h @@ -26,7 +26,7 @@ public: return std::make_shared(table_id, columns, std::move(pipes), to_stage); } std::string getName() const override { return "Blocks"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const override { return to_stage; } + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const override { return to_stage; } Pipes read( const Names & /*column_names*/, diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 487fe6bcb9d..df9fcbf750e 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -54,7 +54,7 @@ public: std::string getName() const override { return "Buffer"; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr &) const override; Pipes read( const Names & column_names, diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 8eea29ba8d6..7f533d9588b 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -66,7 +66,7 @@ public: bool isRemote() const override { return true; } - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr &) const override; Pipes read( const Names & column_names, diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 34704644b00..6284f791f4f 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -59,7 +59,7 @@ public: void checkTableCanBeDropped() const override; void checkPartitionCanBeDropped(const ASTPtr & partition) override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr & = nullptr) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ASTPtr &) const override; StoragePtr getTargetTable() const; StoragePtr tryGetTargetTable() const; diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index 5d7f527d8b6..1d2df3cb9ce 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -31,7 +31,7 @@ public: NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; - QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr & = nullptr) const override; + QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const override; Pipes read( const Names & column_names, From 445d7509388f71e0225f5aef3a4309193811a6bc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 24 Mar 2020 18:56:46 +0300 Subject: [PATCH 067/147] fix build --- dbms/src/Interpreters/evaluateConstantExpression.cpp | 6 +++--- dbms/src/Parsers/ASTLiteral.cpp | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index a99f6cf1b3a..417dccadb3d 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -116,10 +116,10 @@ namespace if (name == identifier->name) { ColumnWithTypeAndName column; - Field value = convertFieldToType(literal->value, *type); - if (!literal->value.isNull() && value.isNull()) + Field converted = convertFieldToType(value, *type); + if (converted.isNull()) return {}; - column.column = type->createColumnConst(1, value); + column.column = type->createColumnConst(1, converted); column.name = name; column.type = type; return {{std::move(column)}}; diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 1a75ad019e5..92d57687426 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -17,13 +17,13 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { /// 100 - just arbitrary value. - constexpr auto MIN_ELEMENTS_FOR_HASHING = 100; + constexpr auto min_elements_for_hashing = 100; /// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) auto type = value.getType(); - if ((type == Field::Types::Array && value.get().size() > MIN_ELEMENTS_FOR_HASHING) - || (type == Field::Types::Tuple && value.get().size() > MIN_ELEMENTS_FOR_HASHING)) + if ((type == Field::Types::Array && value.get().size() > min_elements_for_hashing) + || (type == Field::Types::Tuple && value.get().size() > min_elements_for_hashing)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); From 72327d4836fb6dfa00a89b27d17aa9cf9ee6ca10 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 26 Mar 2020 13:33:53 +0300 Subject: [PATCH 068/147] Better check for supported instructions --- base/daemon/BaseDaemon.cpp | 128 ------------------------------------- dbms/programs/main.cpp | 124 +++++++++++++++++++++++++++++++++++ 2 files changed, 124 insertions(+), 128 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index c3750f6feeb..154885289b7 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -366,136 +366,8 @@ void BaseDaemon::reloadConfiguration() } -namespace -{ - -enum class InstructionFail -{ - NONE = 0, - SSE3 = 1, - SSSE3 = 2, - SSE4_1 = 3, - SSE4_2 = 4, - AVX = 5, - AVX2 = 6, - AVX512 = 7 -}; - -std::string instructionFailToString(InstructionFail fail) -{ - switch (fail) - { - case InstructionFail::NONE: - return "NONE"; - case InstructionFail::SSE3: - return "SSE3"; - case InstructionFail::SSSE3: - return "SSSE3"; - case InstructionFail::SSE4_1: - return "SSE4.1"; - case InstructionFail::SSE4_2: - return "SSE4.2"; - case InstructionFail::AVX: - return "AVX"; - case InstructionFail::AVX2: - return "AVX2"; - case InstructionFail::AVX512: - return "AVX512"; - } - __builtin_unreachable(); -} - - -sigjmp_buf jmpbuf; - -void sigIllCheckHandler(int, siginfo_t *, void *) -{ - siglongjmp(jmpbuf, 1); -} - -/// Check if necessary sse extensions are available by trying to execute some sse instructions. -/// If instruction is unavailable, SIGILL will be sent by kernel. -void checkRequiredInstructionsImpl(volatile InstructionFail & fail) -{ -#if __SSE3__ - fail = InstructionFail::SSE3; - __asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0"); -#endif - -#if __SSSE3__ - fail = InstructionFail::SSSE3; - __asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0"); - -#endif - -#if __SSE4_1__ - fail = InstructionFail::SSE4_1; - __asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0"); -#endif - -#if __SSE4_2__ - fail = InstructionFail::SSE4_2; - __asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0"); -#endif - -#if __AVX__ - fail = InstructionFail::AVX; - __asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0"); -#endif - -#if __AVX2__ - fail = InstructionFail::AVX2; - __asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0"); -#endif - -#if __AVX512__ - fail = InstructionFail::AVX512; - __asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0"); -#endif - - fail = InstructionFail::NONE; -} - -/// Check SSE and others instructions availability -/// Calls exit on fail -void checkRequiredInstructions() -{ - struct sigaction sa{}; - struct sigaction sa_old{}; - sa.sa_sigaction = sigIllCheckHandler; - sa.sa_flags = SA_SIGINFO; - auto signal = SIGILL; - if (sigemptyset(&sa.sa_mask) != 0 - || sigaddset(&sa.sa_mask, signal) != 0 - || sigaction(signal, &sa, &sa_old) != 0) - { - std::cerr << "Can not set signal handler\n"; - exit(1); - } - - volatile InstructionFail fail = InstructionFail::NONE; - - if (sigsetjmp(jmpbuf, 1)) - { - std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n"; - exit(1); - } - - checkRequiredInstructionsImpl(fail); - - if (sigaction(signal, &sa_old, nullptr)) - { - std::cerr << "Can not set signal handler\n"; - exit(1); - } -} - -} - - BaseDaemon::BaseDaemon() { - checkRequiredInstructions(); } diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 17518ad4e37..7955069aab2 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -118,6 +118,128 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name)); } + +enum class InstructionFail +{ + NONE = 0, + SSE3 = 1, + SSSE3 = 2, + SSE4_1 = 3, + SSE4_2 = 4, + AVX = 5, + AVX2 = 6, + AVX512 = 7 +}; + +std::string instructionFailToString(InstructionFail fail) +{ + switch (fail) + { + case InstructionFail::NONE: + return "NONE"; + case InstructionFail::SSE3: + return "SSE3"; + case InstructionFail::SSSE3: + return "SSSE3"; + case InstructionFail::SSE4_1: + return "SSE4.1"; + case InstructionFail::SSE4_2: + return "SSE4.2"; + case InstructionFail::AVX: + return "AVX"; + case InstructionFail::AVX2: + return "AVX2"; + case InstructionFail::AVX512: + return "AVX512"; + } + __builtin_unreachable(); +} + + +sigjmp_buf jmpbuf; + +void sigIllCheckHandler(int, siginfo_t *, void *) +{ + siglongjmp(jmpbuf, 1); +} + +/// Check if necessary sse extensions are available by trying to execute some sse instructions. +/// If instruction is unavailable, SIGILL will be sent by kernel. +void checkRequiredInstructionsImpl(volatile InstructionFail & fail) +{ +#if __SSE3__ + fail = InstructionFail::SSE3; + __asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0"); +#endif + +#if __SSSE3__ + fail = InstructionFail::SSSE3; + __asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0"); + +#endif + +#if __SSE4_1__ + fail = InstructionFail::SSE4_1; + __asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0"); +#endif + +#if __SSE4_2__ + fail = InstructionFail::SSE4_2; + __asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0"); +#endif + +#if __AVX__ + fail = InstructionFail::AVX; + __asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0"); +#endif + +#if __AVX2__ + fail = InstructionFail::AVX2; + __asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0"); +#endif + +#if __AVX512__ + fail = InstructionFail::AVX512; + __asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0"); +#endif + + fail = InstructionFail::NONE; +} + +/// Check SSE and others instructions availability +/// Calls exit on fail +void checkRequiredInstructions() +{ + struct sigaction sa{}; + struct sigaction sa_old{}; + sa.sa_sigaction = sigIllCheckHandler; + sa.sa_flags = SA_SIGINFO; + auto signal = SIGILL; + if (sigemptyset(&sa.sa_mask) != 0 + || sigaddset(&sa.sa_mask, signal) != 0 + || sigaction(signal, &sa, &sa_old) != 0) + { + std::cerr << "Can not set signal handler\n"; + exit(1); + } + + volatile InstructionFail fail = InstructionFail::NONE; + + if (sigsetjmp(jmpbuf, 1)) + { + std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n"; + exit(1); + } + + checkRequiredInstructionsImpl(fail); + + if (sigaction(signal, &sa_old, nullptr)) + { + std::cerr << "Can not set signal handler\n"; + exit(1); + } +} + } @@ -131,6 +253,8 @@ bool inside_main = false; int main(int argc_, char ** argv_) { + checkRequiredInstructions(); + inside_main = true; SCOPE_EXIT({ inside_main = false; }); From 68726fe2a273bc8c2ff1fa71c96a321babaa200a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 26 Mar 2020 19:58:50 +0300 Subject: [PATCH 069/147] Better check for available instructions --- dbms/programs/main.cpp | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 7955069aab2..109d25a5f7d 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -1,3 +1,6 @@ +#include +#include + #include #include #include @@ -158,7 +161,7 @@ std::string instructionFailToString(InstructionFail fail) sigjmp_buf jmpbuf; -void sigIllCheckHandler(int, siginfo_t *, void *) +[[noreturn]] void sigIllCheckHandler(int, siginfo_t *, void *) { siglongjmp(jmpbuf, 1); } @@ -167,38 +170,38 @@ void sigIllCheckHandler(int, siginfo_t *, void *) /// If instruction is unavailable, SIGILL will be sent by kernel. void checkRequiredInstructionsImpl(volatile InstructionFail & fail) { -#if __SSE3__ +#if defined(__SSE3__) fail = InstructionFail::SSE3; __asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0"); #endif -#if __SSSE3__ +#if defined(__SSSE3__) fail = InstructionFail::SSSE3; __asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0"); #endif -#if __SSE4_1__ +#if defined(__SSE4_1__) fail = InstructionFail::SSE4_1; __asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0"); #endif -#if __SSE4_2__ +#if defined(__SSE4_2__) fail = InstructionFail::SSE4_2; __asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0"); #endif -#if __AVX__ +#if defined(__AVX__) fail = InstructionFail::AVX; __asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0"); #endif -#if __AVX2__ +#if defined(__AVX2__) fail = InstructionFail::AVX2; __asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0"); #endif -#if __AVX512__ +#if defined(__AVX512__) fail = InstructionFail::AVX512; __asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0"); #endif @@ -208,6 +211,7 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) /// Check SSE and others instructions availability /// Calls exit on fail +/// This function must be called as early inside main as possible. void checkRequiredInstructions() { struct sigaction sa{}; From 949c0f2d45931c5fd009ef6ba6c0c9d03d91a2ec Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 26 Mar 2020 20:13:14 +0300 Subject: [PATCH 070/147] better tuples parsing in format 'VALUES' --- .../Impl/ConstantExpressionTemplate.cpp | 196 ++++++++++-------- 1 file changed, 107 insertions(+), 89 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 18859ea40fd..31357b70ead 100644 --- a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -31,13 +31,27 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } + struct SpecialParserType { - bool is_array = false; - bool is_nullable = false; - Field::Types::Which nested_type = Field::Types::Which::String; + SpecialParserType() = default; + SpecialParserType(Field::Types::Which main_type_) : main_type(main_type_) {} - bool useDefaultParser() const { return nested_type == Field::Types::Which::String; } + Field::Types::Which main_type = Field::Types::String; + bool is_nullable = false; + bool is_array = false; + bool is_tuple = false; + /// Type and nullability + std::vector> nested_types; + + bool useDefaultParser() const + { + return main_type == Field::Types::String || (!nested_types.empty() + && std::all_of( + nested_types.begin(), + nested_types.end(), + [](const auto & type) { return type.first == Field::Types::String; })); + } }; struct LiteralInfo @@ -55,6 +69,54 @@ struct LiteralInfo SpecialParserType special_parser; }; +static void fillLiteralInfo(DataTypes & nested_types, LiteralInfo & info) +{ + size_t elements_num = nested_types.size(); + info.special_parser.nested_types.reserve(elements_num); + + for (auto nested_type : nested_types) + { + /// It can be Array(Nullable(nested_type)) or Tuple(..., Nullable(nested_type), ...) + bool is_nullable = false; + if (auto nullable = dynamic_cast(nested_type.get())) + { + nested_type = nullable->getNestedType(); + is_nullable = true; + } + + WhichDataType type_info{nested_type}; + Field::Types::Which field_type; + + /// Promote integers to 64 bit types + if (type_info.isNativeUInt()) + { + nested_type = std::make_shared(); + field_type = Field::Types::UInt64; + } + else if (type_info.isNativeInt()) + { + nested_type = std::make_shared(); + field_type = Field::Types::Int64; + } + else if (type_info.isFloat64()) + { + field_type = Field::Types::Float64; + } + else if (type_info.isString()) + { + field_type = Field::Types::String; + } + else + throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug", + ErrorCodes::LOGICAL_ERROR); + + if (is_nullable) + nested_type = std::make_shared(nested_type); + + info.special_parser.nested_types.emplace_back(field_type, is_nullable); + } +} + /// Extracts ASTLiterals from expression, replaces them with ASTIdentifiers where needed /// and deduces data types for dummy columns by field type of literal class ReplaceLiteralsVisitor @@ -138,7 +200,7 @@ private: /// We have to use ParserNumber instead of type->deserializeAsTextQuoted() for arithmetic types /// to check actual type of literal and avoid possible overflow and precision issues. - info.special_parser = SpecialParserType{false, false, field_type}; + info.special_parser = SpecialParserType(field_type); /// Do not use 8, 16 and 32 bit types, so template will match all integers if (field_type == Field::Types::UInt64) @@ -149,69 +211,21 @@ private: info.type = std::make_shared(); else if (field_type == Field::Types::String) info.type = std::make_shared(); - else if (field_type == Field::Types::Array || field_type == Field::Types::Tuple) + else if (field_type == Field::Types::Array) { info.special_parser.is_array = true; info.type = applyVisitor(FieldToDataType(), info.literal->value); - - DataTypePtr nested_type; - if (auto array_type = typeid_cast(info.type.get())) - nested_type = array_type->getNestedType(); - else if (auto tuple_type = typeid_cast(info.type.get())) - nested_type = tuple_type->getElements()[0]; - else - throw Exception("Unexpected type " + info.type->getName(), ErrorCodes::LOGICAL_ERROR); - - /// It can be Array(Nullable(nested_type)) - bool array_of_nullable = false; - if (auto nullable = dynamic_cast(nested_type.get())) - { - nested_type = nullable->getNestedType(); - array_of_nullable = true; - } - - WhichDataType type_info{nested_type}; - /// Promote integers to 64 bit types - if (type_info.isNativeUInt()) - { - nested_type = std::make_shared(); - info.special_parser.nested_type = Field::Types::UInt64; - } - else if (type_info.isNativeInt()) - { - nested_type = std::make_shared(); - info.special_parser.nested_type = Field::Types::Int64; - } - else if (type_info.isFloat64()) - { - info.special_parser.nested_type = Field::Types::Float64; - } - else if (type_info.isString()) - { - info.special_parser.nested_type = Field::Types::String; - } - else - throw Exception("Unexpected literal type inside Array: " + nested_type->getName() + ". It's a bug", - ErrorCodes::LOGICAL_ERROR); - - if (array_of_nullable) - { - nested_type = std::make_shared(nested_type); - info.special_parser.is_nullable = true; - } - - if (field_type == Field::Types::Tuple) - { - const auto & tuple = info.literal->value.get(); - DataTypes elements(tuple.size()); - for (size_t i = 0; i < tuple.size(); ++i) - elements[i] = nested_type; - info.type = std::make_shared(elements); - } - else - { - info.type = std::make_shared(nested_type); - } + DataTypes nested_types = { assert_cast(*info.type).getNestedType() }; + fillLiteralInfo(nested_types, info); + info.type = std::make_shared(nested_types[0]); + } + else if (field_type == Field::Types::Tuple) + { + info.special_parser.is_tuple = true; + info.type = applyVisitor(FieldToDataType(), info.literal->value); + auto nested_types = assert_cast(*info.type).getElements(); + fillLiteralInfo(nested_types, info); + info.type = std::make_shared(nested_types); } else throw Exception(String("Unexpected literal type ") + info.literal->value.getTypeName() + ". It's a bug", @@ -423,7 +437,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co /// If literal does not fit entirely in the buffer, parsing error will happen. /// However, it's possible to deduce new template (or use template from cache) after error like it was template mismatch. - if (type_info.is_array) + if (type_info.is_array || type_info.is_tuple) { /// TODO faster way to check types without using Parsers ParserArrayOfLiterals parser_array; @@ -435,34 +449,38 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co ASTPtr ast; if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected)) return false; + istr.position() = const_cast(iterator->begin); const Field & collection = ast->as().value; auto collection_type = applyVisitor(FieldToDataType(), collection); - DataTypePtr nested_type; - if (auto array_type = typeid_cast(collection_type.get())) - nested_type = array_type->getNestedType(); - else if (auto tuple_type = typeid_cast(collection_type.get())) - nested_type = tuple_type->getElements()[0]; + DataTypes nested_types; + if (type_info.is_array) + nested_types = { assert_cast(*collection_type).getNestedType() }; + else + nested_types = assert_cast(*collection_type).getElements(); - if (!nested_type) - return false; - - if (type_info.is_nullable) - if (auto nullable = dynamic_cast(nested_type.get())) - nested_type = nullable->getNestedType(); - - WhichDataType nested_type_info(nested_type); - if ((nested_type_info.isNativeUInt() && type_info.nested_type == Type::UInt64) || - (nested_type_info.isNativeInt() && type_info.nested_type == Type::Int64) || - (nested_type_info.isFloat64() && type_info.nested_type == Type::Float64)) + for (size_t i = 0; i < nested_types.size(); ++i) { - Field array_same_types = convertFieldToType(collection, *complex_type, nullptr); - columns[column_idx]->insert(array_same_types); - return true; + const auto & [nested_field_type, is_nullable] = type_info.nested_types[i]; + if (is_nullable) + if (auto nullable = dynamic_cast(nested_types[i].get())) + nested_types[i] = nullable->getNestedType(); + + WhichDataType nested_type_info(nested_types[i]); + bool are_types_compatible = + (nested_type_info.isNativeUInt() && nested_field_type == Type::UInt64) || + (nested_type_info.isNativeInt() && nested_field_type == Type::Int64) || + (nested_type_info.isFloat64() && nested_field_type == Type::Float64); + + if (!are_types_compatible) + return false; } - return false; + + Field array_same_types = convertFieldToType(collection, *complex_type, nullptr); + columns[column_idx]->insert(array_same_types); + return true; } else { @@ -499,14 +517,14 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co if (pos_integer == pos_double && errno != ERANGE && (!negative || uint_value <= (1ULL << 63))) { istr.position() += pos_integer - buf; - if (negative && type_info.nested_type == Type::Int64) + if (negative && type_info.main_type == Type::Int64) number = static_cast(-uint_value); - else if (!negative && type_info.nested_type == Type::UInt64) + else if (!negative && type_info.main_type == Type::UInt64) number = uint_value; else return false; } - else if (type_info.nested_type == Type::Float64) + else if (type_info.main_type == Type::Float64) { istr.position() += pos_double - buf; number = float_value; From e4b9fcaddcef359cba7954b45549296bf62d2698 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 26 Mar 2020 20:53:46 +0300 Subject: [PATCH 071/147] Added a test --- ...01103_check_cpu_instructions_at_startup.reference | 4 ++++ .../01103_check_cpu_instructions_at_startup.sh | 12 ++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference create mode 100755 dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh diff --git a/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference b/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference new file mode 100644 index 00000000000..59c88f013dd --- /dev/null +++ b/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference @@ -0,0 +1,4 @@ +Instruction check fail. The CPU does not support SSSE3 instruction set. +Instruction check fail. The CPU does not support SSE4.1 instruction set. +Instruction check fail. The CPU does not support SSE4.2 instruction set. +1 diff --git a/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh new file mode 100755 index 00000000000..4338e3e9ed5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +command=$(command -v ${CLICKHOUSE_LOCAL}) + +qemu-x86_64-static -cpu qemu64 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: +qemu-x86_64-static -cpu qemu64,+ssse3 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: +qemu-x86_64-static -cpu qemu64,+ssse3,+sse4.1 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: +qemu-x86_64-static -cpu qemu64,+ssse3,+sse4.1,+sse4.2 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: + From 6eb98d8c77b78320c75a8450ef625ca0d84ebcad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 26 Mar 2020 20:53:57 +0300 Subject: [PATCH 072/147] Proper check --- dbms/programs/main.cpp | 53 ++++++++++++++++++++++++++++++++---------- 1 file changed, 41 insertions(+), 12 deletions(-) diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 109d25a5f7d..b56341d35c7 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -134,7 +135,7 @@ enum class InstructionFail AVX512 = 7 }; -std::string instructionFailToString(InstructionFail fail) +const char * instructionFailToString(InstructionFail fail) { switch (fail) { @@ -209,9 +210,26 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) fail = InstructionFail::NONE; } -/// Check SSE and others instructions availability -/// Calls exit on fail -/// This function must be called as early inside main as possible. +/// This function is safe to use in static initializers. +void writeError(const char * data, size_t size) +{ + while (size != 0) + { + ssize_t res = ::write(STDERR_FILENO, data, size); + + if ((-1 == res || 0 == res) && errno != EINTR) + _Exit(1); + + if (res > 0) + { + data += res; + size -= res; + } + } +} + +/// Check SSE and others instructions availability. Calls exit on fail. +/// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions. void checkRequiredInstructions() { struct sigaction sa{}; @@ -223,27 +241,40 @@ void checkRequiredInstructions() || sigaddset(&sa.sa_mask, signal) != 0 || sigaction(signal, &sa, &sa_old) != 0) { - std::cerr << "Can not set signal handler\n"; - exit(1); + /// You may wonder about strlen. + /// Typical implementation of strlen is using SSE4.2 or AVX2. + /// But this is not the case because it's compiler builtin and is executed at compile time. + + const char * msg = "Can not set signal handler\n"; + writeError(msg, strlen(msg)); + _Exit(1); } volatile InstructionFail fail = InstructionFail::NONE; if (sigsetjmp(jmpbuf, 1)) { - std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n"; - exit(1); + const char * msg1 = "Instruction check fail. The CPU does not support "; + writeError(msg1, strlen(msg1)); + const char * msg2 = instructionFailToString(fail); + writeError(msg2, strlen(msg2)); + const char * msg3 = " instruction set.\n"; + writeError(msg3, strlen(msg3)); + _Exit(1); } checkRequiredInstructionsImpl(fail); if (sigaction(signal, &sa_old, nullptr)) { - std::cerr << "Can not set signal handler\n"; - exit(1); + const char * msg = "Can not set signal handler\n"; + writeError(msg, strlen(msg)); + _Exit(1); } } +struct Checker { Checker() { checkRequiredInstructions(); } } checker; + } @@ -257,8 +288,6 @@ bool inside_main = false; int main(int argc_, char ** argv_) { - checkRequiredInstructions(); - inside_main = true; SCOPE_EXIT({ inside_main = false; }); From 7ef1064a408ba5a5169b0e3a96c3df769b795dc7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 26 Mar 2020 20:55:11 +0300 Subject: [PATCH 073/147] Update Dockerfiles for tests --- docker/test/stateless/Dockerfile | 3 ++- docker/test/stateless_with_coverage/Dockerfile | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index b5604855b38..2d2025de58b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -30,7 +30,8 @@ RUN apt-get update -y \ lsof \ llvm-9 \ unixodbc \ - wget + wget \ + qemu-user-static RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index 4da82c67c40..f872f87b3f4 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -33,7 +33,8 @@ RUN apt-get update -y \ lsof \ llvm-9 \ unixodbc \ - wget + wget \ + qemu-user-static RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget --quiet -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ From 9e5d82641fb8765758c26bc8d0fc7ea8f29b4a09 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 26 Mar 2020 22:21:31 +0300 Subject: [PATCH 074/147] fix build --- dbms/src/Processors/Executors/PipelineExecutor.h | 1 - .../src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index 20569d974ee..673151bd5eb 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -273,7 +273,6 @@ private: void executeSingleThread(size_t thread_num, size_t num_threads); void finish(); -public: String dumpPipeline() const; }; diff --git a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 31357b70ead..99369ffa26b 100644 --- a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -35,7 +34,7 @@ namespace ErrorCodes struct SpecialParserType { SpecialParserType() = default; - SpecialParserType(Field::Types::Which main_type_) : main_type(main_type_) {} + explicit SpecialParserType(Field::Types::Which main_type_) : main_type(main_type_) {} Field::Types::Which main_type = Field::Types::String; bool is_nullable = false; From f3975e000ed558aacde0fba6629c10c87ae3319f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 27 Mar 2020 00:03:02 +0300 Subject: [PATCH 075/147] Update BaseDaemon.cpp --- base/daemon/BaseDaemon.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 154885289b7..777e188e728 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -366,9 +366,7 @@ void BaseDaemon::reloadConfiguration() } -BaseDaemon::BaseDaemon() -{ -} +BaseDaemon::BaseDaemon() = default; BaseDaemon::~BaseDaemon() From 3e17cf3a9bd1c7bf28f8827f54405633e9144012 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 27 Mar 2020 11:14:20 +0300 Subject: [PATCH 076/147] Update main.cpp --- dbms/programs/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index b56341d35c7..3c16fb64a57 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -167,7 +167,7 @@ sigjmp_buf jmpbuf; siglongjmp(jmpbuf, 1); } -/// Check if necessary sse extensions are available by trying to execute some sse instructions. +/// Check if necessary SSE extensions are available by trying to execute some sse instructions. /// If instruction is unavailable, SIGILL will be sent by kernel. void checkRequiredInstructionsImpl(volatile InstructionFail & fail) { From d3e8335a8da0105cfc1b1d07e79b0fed0ca4451f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 27 Mar 2020 13:53:04 +0300 Subject: [PATCH 077/147] Fix race condition between optimize and drop --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 ++ .../MergeTree/ReplicatedMergeTreeQueue.h | 9 +++ .../ReplicatedMergeTreeRestartingThread.cpp | 1 - .../Storages/StorageReplicatedMergeTree.cpp | 9 ++- .../src/Storages/StorageReplicatedMergeTree.h | 3 - ...103_optimize_drop_race_zookeeper.reference | 0 .../01103_optimize_drop_race_zookeeper.sh | 61 +++++++++++++++++++ 7 files changed, 82 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.reference create mode 100755 dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 73ea2098c71..c0749818ebf 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1549,6 +1549,10 @@ std::vector ReplicatedMergeTreeQueue::getMutationsStatu return result; } +ReplicatedMergeTreeQueue::QueueLocks ReplicatedMergeTreeQueue::lockQueue() +{ + return QueueLocks(state_mutex, pull_logs_to_queue_mutex, update_mutations_mutex); +} ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 22d198b9f19..534978873c2 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -384,6 +384,15 @@ public: std::vector getMutationsStatus() const; void removeCurrentPartsFromMutations(); + + using QueueLocks = std::scoped_lock; + + /// This method locks all important queue mutexes: state_mutex, + /// pull_logs_to_queue and update_mutations_mutex. It should be used only + /// once while we want to shutdown our queue and remove it's task from pool. + /// It's needed because queue itself can trigger it's task handler and in + /// this case race condition is possible. + QueueLocks lockQueue(); }; class ReplicatedMergeTreeMergePredicate diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 37b428e1e47..cd82a865827 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -334,7 +334,6 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown() storage.partial_shutdown_called = true; storage.partial_shutdown_event.set(); - storage.alter_query_event->set(); storage.replica_is_active_node = nullptr; LOG_TRACE(log, "Waiting for threads to finish"); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 68cc98cb1b9..b8e42836c8c 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2882,7 +2882,14 @@ void StorageReplicatedMergeTree::shutdown() if (queue_task_handle) global_context.getBackgroundPool().removeTask(queue_task_handle); - queue_task_handle.reset(); + + { + /// Queue can trigger queue_task_handle itself. So we ensure that all + /// queue processes finished and after that reset queue_task_handle. + auto lock = queue.lockQueue(); + queue_task_handle.reset(); + } + if (move_parts_task_handle) global_context.getBackgroundMovePool().removeTask(move_parts_task_handle); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index f6483baf353..1b92310b39d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -284,9 +284,6 @@ private: /// A thread that processes reconnection to ZooKeeper when the session expires. ReplicatedMergeTreeRestartingThread restarting_thread; - /// An event that awakens `alter` method from waiting for the completion of the ALTER query. - zkutil::EventPtr alter_query_event = std::make_shared(); - /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; diff --git a/dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.reference b/dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh b/dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh new file mode 100755 index 00000000000..82df41d6223 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +function thread1() +{ + while true; do + $CLICKHOUSE_CLIENT -q "INSERT INTO concurrent_optimize_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(10000)"; + done +} + + +function thread2() +{ + while true; do + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE concurrent_optimize_table FINAL"; + sleep 0.$RANDOM; + done +} + +function thread3() +{ + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS concurrent_optimize_table; + CREATE TABLE concurrent_optimize_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_optimize_table', '1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + done +} + + +export -f thread1; +export -f thread2; +export -f thread3; + + +TIMEOUT=15 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & + +wait + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_optimize_table" From 4003dcd12ef7da11d05ffaebb63a1118e8efdc5b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 27 Mar 2020 17:51:14 +0300 Subject: [PATCH 078/147] Better storage def in flaky test --- .../00992_system_parts_race_condition_zookeeper.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh index 7dbfda90e41..846156f5b6d 100755 --- a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh @@ -9,8 +9,8 @@ $CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS alter_table; DROP TABLE IF EXISTS alter_table2; - CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_DATABASE.alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1; - CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_DATABASE.alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1 + CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_DATABASE.alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; + CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_DATABASE.alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 " function thread1() From c0e3564ff58580893cfd314aafefda7b4c154672 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 27 Mar 2020 23:10:27 +0800 Subject: [PATCH 079/147] Fix scalar optimization in ALTER --- dbms/src/Interpreters/MutationsInterpreter.cpp | 4 ++++ dbms/src/Interpreters/MutationsInterpreter.h | 2 +- .../01220_scalar_optimization_in_alter.reference | 0 .../0_stateless/01220_scalar_optimization_in_alter.sql | 9 +++++++++ 4 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.reference create mode 100644 dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.sql diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 311165066c9..669b72c6317 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -558,6 +558,10 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & all_asts->children.push_back(std::make_shared(column)); auto syntax_result = SyntaxAnalyzer(context).analyze(all_asts, all_columns); + if (context.hasQueryContext()) + for (const auto & it : syntax_result->getScalars()) + context.getQueryContext().addScalar(it.first, it.second); + stage.analyzer = std::make_unique(all_asts, syntax_result, context); ExpressionActionsChain & actions_chain = stage.expressions_chain; diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index a36430e201b..9b12ce79e91 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -47,7 +47,7 @@ private: StoragePtr storage; MutationCommands commands; - const Context & context; + Context context; bool can_execute; ASTPtr mutation_ast; diff --git a/dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.reference b/dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.sql b/dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.sql new file mode 100644 index 00000000000..de92b29ce5d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01220_scalar_optimization_in_alter.sql @@ -0,0 +1,9 @@ +drop table if exists cdp_segments; +drop table if exists cdp_customers; + +create table cdp_segments (seg_id String, mid_seqs AggregateFunction(groupBitmap, UInt32)) engine=ReplacingMergeTree() order by (seg_id); +create table cdp_customers (mid String, mid_seq UInt32) engine=ReplacingMergeTree() order by (mid_seq); +alter table cdp_segments update mid_seqs = bitmapOr(mid_seqs, (select groupBitmapState(mid_seq) from cdp_customers where mid in ('6bf3c2ee-2b33-3030-9dc2-25c6c618d141'))) where seg_id = '1234567890'; + +drop table cdp_segments; +drop table cdp_customers; From 77fe0b7f0622e716f75d20ec1b4f9a5a5117f2ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 27 Mar 2020 18:23:11 +0300 Subject: [PATCH 080/147] turn on polymorphic parts for log tables --- dbms/src/Interpreters/SystemLog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index e307394e361..51a734b083f 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -57,7 +57,8 @@ std::shared_ptr createSystemLog( else { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); - engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024"; + engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)" + "SETTINGS index_granularity = 1024, min_bytes_for_wide_part = 10485760"; /// Use polymorphic parts for log tables by default } size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); From f08f16d0eaa0c6ff89656598b7e4f6f8034f6dc5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 27 Mar 2020 18:35:26 +0300 Subject: [PATCH 081/147] Get rid of backticks in headers and outdated comment --- .../query_language/functions/comparison_functions.md | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/en/query_language/functions/comparison_functions.md b/docs/en/query_language/functions/comparison_functions.md index 4505cc8cbf1..6865fce5ef0 100644 --- a/docs/en/query_language/functions/comparison_functions.md +++ b/docs/en/query_language/functions/comparison_functions.md @@ -15,18 +15,16 @@ For example, you can’t compare a date with a string. You have to use a functio Strings are compared by bytes. A shorter string is smaller than all strings that start with it and that contain at least one more character. -Note. Up until version 1.1.54134, signed and unsigned numbers were compared the same way as in C++. In other words, you could get an incorrect result in cases like SELECT 9223372036854775807 \> -1. This behavior changed in version 1.1.54134 and is now mathematically correct. - ## equals, a = b and a == b operator {#function-equals} -## notEquals, a ! operator= b and a `<>` b {#function-notequals} +## notEquals, a ! operator= b and a <> b {#function-notequals} -## less, `< operator` {#function-less} +## less, < operator {#function-less} -## greater, `> operator` {#function-greater} +## greater, > operator {#function-greater} -## lessOrEquals, `<= operator` {#function-lessorequals} +## lessOrEquals, <= operator {#function-lessorequals} -## greaterOrEquals, `>= operator` {#function-greaterorequals} +## greaterOrEquals, >= operator {#function-greaterorequals} [Original article](https://clickhouse.tech/docs/en/query_language/functions/comparison_functions/) From 318d7f45f01d37026f904abbad82045f7fc3caf4 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 27 Mar 2020 23:10:03 +0300 Subject: [PATCH 082/147] Fix errors --- .../Formats/Impl/RegexpRowInputFormat.cpp | 60 ++++++++++--------- .../Formats/Impl/RegexpRowInputFormat.h | 23 +++---- 2 files changed, 44 insertions(+), 39 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index e26255deae4..f7afd5848c7 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { @@ -33,17 +32,17 @@ RegexpRowInputFormat::RegexpRowInputFormat( field_format = stringToFormat(format_settings_.regexp.escaping_rule); } -RegexpRowInputFormat::FieldFormat RegexpRowInputFormat::stringToFormat(const String & format) +RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const String & format) { if (format == "Escaped") - return FieldFormat::Escaped; + return ColumnFormat::Escaped; if (format == "Quoted") - return FieldFormat::Quoted; + return ColumnFormat::Quoted; if (format == "CSV") - return FieldFormat::Csv; + return ColumnFormat::Csv; if (format == "JSON") - return FieldFormat::Json; - throw Exception("Unknown field format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); + return ColumnFormat::Json; + throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); } bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) @@ -51,40 +50,42 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) const auto & type = getPort().getHeader().getByPosition(index).type; bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); bool read = true; - ReadBuffer field_buf(matched_fields[index].data(), matched_fields[index].size(), 0); + ReadBuffer field_buf(const_cast(matched_fields[index].data()), matched_fields[index].size(), 0); try { switch (field_format) { - case FieldFormat::Escaped: + case ColumnFormat::Escaped: if (parse_as_nullable) read = DataTypeNullable::deserializeTextEscaped(*columns[index], field_buf, format_settings, type); else type->deserializeAsTextEscaped(*columns[index], field_buf, format_settings); break; - case FieldFormat::Quoted: + case ColumnFormat::Quoted: if (parse_as_nullable) read = DataTypeNullable::deserializeTextQuoted(*columns[index], field_buf, format_settings, type); else type->deserializeAsTextQuoted(*columns[index], field_buf, format_settings); break; - case FieldFormat::Csv: + case ColumnFormat::Csv: if (parse_as_nullable) read = DataTypeNullable::deserializeTextCSV(*columns[index], field_buf, format_settings, type); else type->deserializeAsTextCSV(*columns[index], field_buf, format_settings); break; - case FieldFormat::Json: + case ColumnFormat::Json: if (parse_as_nullable) read = DataTypeNullable::deserializeTextJSON(*columns[index], field_buf, format_settings, type); else type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); break; + default: + break; } } catch (Exception & e) { - e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); + e.addMessage("(while read the value of column " + getPort().getHeader().getByPosition(index).name + ")"); throw; } return read; @@ -111,11 +112,11 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & size_t line_size = 0; - while (!buf.eof() && *buf.position() != '\n' && *buf.position() != '\r') - { - ++buf.position(); - ++line_size; - } + do { + char *pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end()); + line_size += pos - buf.position(); + buf.position() = pos; + } while (buf.position() == buf.buffer().end() && !buf.eof()); buf.makeContinuousMemoryFromCheckpointToPos(); buf.rollbackToCheckpoint(); @@ -135,12 +136,9 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & buf.position() += line_size; - // Two sequential increments are needed to support DOS-style newline ("\r\n"). - if (!buf.eof() && *buf.position() == '\r') - ++buf.position(); - - if (!buf.eof() && *buf.position() == '\n') - ++buf.position(); + checkChar('\r', buf); + if (!buf.eof() && !checkChar('\n', buf)) + throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA); return true; } @@ -160,21 +158,27 @@ void registerInputFormatProcessorRegexp(FormatFactory & factory) static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { char * pos = in.position(); + bool need_more_data = true; - while (loadAtPosition(in, memory, pos) && (memory.size() + static_cast(pos - in.position()) >= min_chunk_size)) + while (loadAtPosition(in, memory, pos) && need_more_data) { pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end()); if (pos == in.buffer().end()) continue; // Support DOS-style newline ("\r\n") - if (*pos++ == '\r') + if (*pos == '\r') { + ++pos; if (pos == in.buffer().end()) loadAtPosition(in, memory, pos); - if (*pos == '\n') - ++pos; } + + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + + ++pos; + } saveUpToPosition(in, memory, pos); diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 218bb71055f..de1520eba8e 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -8,15 +9,23 @@ #include #include #include +#include namespace DB { class ReadBuffer; +/// Regexp input format. +/// This format applies regular expression from format_regexp setting for every line of file +/// (the lines must be separated by newline character ('\n') or DOS-style newline ("\r\n")). +/// Every matched subpattern will be parsed with the method of corresponding data type +/// (according to format_regexp_escaping_rule setting). If the regexp did not match the line, +/// if format_regexp_skip_unmatched is 1, the line is silently skipped, if the setting is 0, exception will be thrown. class RegexpRowInputFormat : public IRowInputFormat { + using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; public: RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); @@ -25,25 +34,17 @@ public: bool readRow(MutableColumns & columns, RowReadExtension & ext) override; private: - enum class FieldFormat - { - Escaped, - Quoted, - Csv, - Json, - }; - bool readField(size_t index, MutableColumns & columns); void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); - FieldFormat stringToFormat(const String & format); + ColumnFormat stringToFormat(const String & format); PeekableReadBuffer buf; const FormatSettings format_settings; - FieldFormat field_format; + ColumnFormat field_format; RE2 regexp; // The vector of fields extracted from line using regexp. - std::vector matched_fields; + std::vector matched_fields; // These two vectors are needed to use RE2::FullMatchN (function for extracting fields). std::vector re2_arguments; std::vector re2_arguments_ptrs; From b1bd11f9623054af050c0bfe70115d5b1cdcae31 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 27 Mar 2020 23:12:14 +0300 Subject: [PATCH 083/147] Rename column names in local distributed_product_mode (#9891) --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 - .../InJoinSubqueriesPreprocessor.cpp | 17 +++- .../InJoinSubqueriesPreprocessor.h | 7 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 + dbms/src/Interpreters/JoinedTables.cpp | 95 +++++++++++++++++++ dbms/src/Interpreters/JoinedTables.h | 2 + dbms/src/Interpreters/SyntaxAnalyzer.cpp | 4 - .../tests/in_join_subqueries_preprocessor.cpp | 3 +- ...roduct_mode_local_column_renames.reference | 12 +++ ...uted_product_mode_local_column_renames.sql | 87 +++++++++++++++++ .../01104_distributed_numbers_test.reference | 4 + .../01104_distributed_numbers_test.sql | 12 +++ .../01104_distributed_one_test.reference | 6 ++ .../01104_distributed_one_test.sql | 18 ++++ 14 files changed, 259 insertions(+), 11 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.reference create mode 100644 dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.sql create mode 100644 dbms/tests/queries/0_stateless/01104_distributed_numbers_test.reference create mode 100644 dbms/tests/queries/0_stateless/01104_distributed_numbers_test.sql create mode 100644 dbms/tests/queries/0_stateless/01104_distributed_one_test.reference create mode 100644 dbms/tests/queries/0_stateless/01104_distributed_one_test.sql diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 85a2dd2c3f8..d1be66df217 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -24,7 +24,6 @@ #include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 389d6c825b0..f40e91e7dcd 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -38,6 +38,7 @@ struct NonGlobalTableData const CheckShardsAndTables & checker; const Context & context; + std::vector & renamed_tables; ASTFunction * function = nullptr; ASTTableJoin * table_join = nullptr; @@ -95,10 +96,11 @@ private: String alias = database_and_table->tryGetAlias(); if (alias.empty()) - throw Exception("Distributed table should have an alias when distributed_product_mode set to local.", + throw Exception("Distributed table should have an alias when distributed_product_mode set to local", ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED); auto & identifier = database_and_table->as(); + renamed_tables.emplace_back(identifier.clone()); identifier.resetTable(database, table); } else @@ -118,6 +120,7 @@ public: { const CheckShardsAndTables & checker; const Context & context; + std::vector>> & renamed_tables; }; static void visit(ASTPtr & node, Data & data) @@ -148,8 +151,11 @@ private: if (node.name == "in" || node.name == "notIn") { auto & subquery = node.arguments->children.at(1); - NonGlobalTableVisitor::Data table_data{data.checker, data.context, &node, nullptr}; + std::vector renamed; + NonGlobalTableVisitor::Data table_data{data.checker, data.context, renamed, &node, nullptr}; NonGlobalTableVisitor(table_data).visit(subquery); + if (!renamed.empty()) + data.renamed_tables.emplace_back(subquery, std::move(renamed)); } } @@ -163,8 +169,11 @@ private: { if (auto & subquery = node.table_expression->as()->subquery) { - NonGlobalTableVisitor::Data table_data{data.checker, data.context, nullptr, table_join}; + std::vector renamed; + NonGlobalTableVisitor::Data table_data{data.checker, data.context, renamed, nullptr, table_join}; NonGlobalTableVisitor(table_data).visit(subquery); + if (!renamed.empty()) + data.renamed_tables.emplace_back(subquery, std::move(renamed)); } } } @@ -208,7 +217,7 @@ void InJoinSubqueriesPreprocessor::visit(ASTPtr & ast) const return; } - NonGlobalSubqueryVisitor::Data visitor_data{*checker, context}; + NonGlobalSubqueryVisitor::Data visitor_data{*checker, context, renamed_tables}; NonGlobalSubqueryVisitor(visitor_data).visit(ast); } diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h index ff39d812dee..5aa9cfbcadf 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h @@ -35,6 +35,8 @@ class Context; class InJoinSubqueriesPreprocessor { public: + using SubqueryTables = std::vector>>; /// {subquery, renamed_tables} + struct CheckShardsAndTables { using Ptr = std::unique_ptr; @@ -45,8 +47,10 @@ public: virtual ~CheckShardsAndTables() {} }; - InJoinSubqueriesPreprocessor(const Context & context_, CheckShardsAndTables::Ptr _checker = std::make_unique()) + InJoinSubqueriesPreprocessor(const Context & context_, SubqueryTables & renamed_tables_, + CheckShardsAndTables::Ptr _checker = std::make_unique()) : context(context_) + , renamed_tables(renamed_tables_) , checker(std::move(_checker)) {} @@ -54,6 +58,7 @@ public: private: const Context & context; + SubqueryTables & renamed_tables; CheckShardsAndTables::Ptr checker; }; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 4fe83afa48d..db1894026d8 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -301,6 +301,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = interpreter_subquery->getSampleBlock(); } + joined_tables.rewriteDistributedInAndJoins(query_ptr); + max_streams = settings.max_threads; ASTSelectQuery & query = getSelectQuery(); diff --git a/dbms/src/Interpreters/JoinedTables.cpp b/dbms/src/Interpreters/JoinedTables.cpp index beec338f9bf..48e763ffb19 100644 --- a/dbms/src/Interpreters/JoinedTables.cpp +++ b/dbms/src/Interpreters/JoinedTables.cpp @@ -1,12 +1,18 @@ #include #include #include +#include +#include +#include #include #include #include #include #include +#include #include +#include +#include namespace DB { @@ -14,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int ALIAS_REQUIRED; + extern const int AMBIGUOUS_COLUMN_NAME; } namespace @@ -32,6 +39,71 @@ void checkTablesWithColumns(const std::vector & tables_with_columns, const Co } } +class RenameQualifiedIdentifiersMatcher +{ +public: + using Data = const std::vector; + + static void visit(ASTPtr & ast, Data & data) + { + if (auto * t = ast->as()) + visit(*t, ast, data); + if (auto * node = ast->as()) + visit(*node, ast, data); + } + + static bool needChildVisit(ASTPtr & node, const ASTPtr & child) + { + if (node->as() || + node->as() || + child->as()) + return false; // NOLINT + return true; + } + +private: + static void visit(ASTIdentifier & identifier, ASTPtr &, Data & data) + { + if (identifier.isShort()) + return; + + bool rewritten = false; + for (auto & table : data) + { + /// Table has an alias. We do not need to rewrite qualified names with table alias (match == ColumnMatch::TableName). + auto match = IdentifierSemantic::canReferColumnToTable(identifier, table); + if (match == IdentifierSemantic::ColumnMatch::AliasedTableName || + match == IdentifierSemantic::ColumnMatch::DbAndTable) + { + if (rewritten) + throw Exception("Failed to rewrite distributed table names. Ambiguous column '" + identifier.name + "'", + ErrorCodes::AMBIGUOUS_COLUMN_NAME); + /// Table has an alias. So we set a new name qualified by table alias. + IdentifierSemantic::setColumnLongName(identifier, table); + rewritten = true; + } + } + } + + static void visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data) + { + ASTIdentifier & identifier = *node.children[0]->as(); + bool rewritten = false; + for (auto & table : data) + { + if (identifier.name == table.table) + { + if (rewritten) + throw Exception("Failed to rewrite distributed table. Ambiguous column '" + identifier.name + "'", + ErrorCodes::AMBIGUOUS_COLUMN_NAME); + identifier.setShortName(table.alias); + rewritten = true; + } + } + } +}; +using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitor; + } JoinedTables::JoinedTables(Context && context_, const ASTSelectQuery & select_query) @@ -114,4 +186,27 @@ void JoinedTables::makeFakeTable(StoragePtr storage, const Block & source_header tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, source_header.getNamesAndTypesList()); } +void JoinedTables::rewriteDistributedInAndJoins(ASTPtr & query) +{ + /// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting. + InJoinSubqueriesPreprocessor::SubqueryTables renamed_tables; + InJoinSubqueriesPreprocessor(context, renamed_tables).visit(query); + + String database; + if (!renamed_tables.empty()) + database = context.getCurrentDatabase(); + + for (auto & [subquery, ast_tables] : renamed_tables) + { + std::vector renamed; + renamed.reserve(ast_tables.size()); + for (auto & ast : ast_tables) + renamed.emplace_back(DatabaseAndTableWithAlias(*ast->as(), database)); + + /// Change qualified column names in distributed subqueries using table aliases. + RenameQualifiedIdentifiersVisitor::Data data(renamed); + RenameQualifiedIdentifiersVisitor(data).visit(subquery); + } +} + } diff --git a/dbms/src/Interpreters/JoinedTables.h b/dbms/src/Interpreters/JoinedTables.h index f1940366ef5..66b3c8de609 100644 --- a/dbms/src/Interpreters/JoinedTables.h +++ b/dbms/src/Interpreters/JoinedTables.h @@ -37,6 +37,8 @@ public: const StorageID & leftTableID() const { return table_id; } + void rewriteDistributedInAndJoins(ASTPtr & query); + std::unique_ptr makeLeftTableSubquery(const SelectQueryOptions & select_options); private: diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 7338487c5e8..f93d11fa1da 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -819,9 +818,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( translateQualifiedNames(query, *select_query, source_columns_set, tables_with_column_names); - /// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting. - InJoinSubqueriesPreprocessor(context).visit(query); - /// Optimizes logical expressions. LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 9a6d7ca4162..9a17f03f32a 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -1181,7 +1181,8 @@ TestResult check(const TestEntry & entry) try { - DB::InJoinSubqueriesPreprocessor(context, std::make_unique()).visit(ast_input); + DB::InJoinSubqueriesPreprocessor::SubqueryTables renamed; + DB::InJoinSubqueriesPreprocessor(context, renamed, std::make_unique()).visit(ast_input); } catch (const DB::Exception & ex) { diff --git a/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.reference b/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.reference new file mode 100644 index 00000000000..53f7332cffb --- /dev/null +++ b/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.reference @@ -0,0 +1,12 @@ +42 +42 +42 +42 +42 +42 +42 +42 +42 +42 +42 +42 diff --git a/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.sql b/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.sql new file mode 100644 index 00000000000..32655420a27 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01103_distributed_product_mode_local_column_renames.sql @@ -0,0 +1,87 @@ +CREATE DATABASE IF NOT EXISTS test_01103; +USE test_01103; + +DROP TABLE IF EXISTS t1_shard; +DROP TABLE IF EXISTS t2_shard; +DROP TABLE IF EXISTS t1_distr; +DROP TABLE IF EXISTS t2_distr; + +create table t1_shard (id Int32) engine MergeTree order by id; +create table t2_shard (id Int32) engine MergeTree order by id; + +create table t1_distr as t1_shard engine Distributed(test_cluster_two_shards_localhost, test_01103, t1_shard, id); +create table t2_distr as t2_shard engine Distributed(test_cluster_two_shards_localhost, test_01103, t2_shard, id); + +insert into t1_shard values (42); +insert into t2_shard values (42); + +SET distributed_product_mode = 'local'; + +select d0.id +from t1_distr d0 +where d0.id in +( + select d1.id + from t1_distr as d1 + inner join t2_distr as d2 on d1.id = d2.id + where d1.id > 0 + order by d1.id +); + +select t1_distr.id +from t1_distr +where t1_distr.id in +( + select t1_distr.id + from t1_distr as d1 + inner join t2_distr as d2 on t1_distr.id = t2_distr.id + where t1_distr.id > 0 + order by t1_distr.id +); + +select test_01103.t1_distr.id +from test_01103.t1_distr +where test_01103.t1_distr.id in +( + select test_01103.t1_distr.id + from test_01103.t1_distr as d1 + inner join test_01103.t2_distr as d2 on test_01103.t1_distr.id = test_01103.t2_distr.id + where test_01103.t1_distr.id > 0 + order by test_01103.t1_distr.id +); + +select d0.id +from t1_distr d0 +join ( + select d1.id + from t1_distr as d1 + inner join t2_distr as d2 on d1.id = d2.id + where d1.id > 0 + order by d1.id +) s0 using id; + +select t1_distr.id +from t1_distr +join ( + select t1_distr.id + from t1_distr as d1 + inner join t2_distr as d2 on t1_distr.id = t2_distr.id + where t1_distr.id > 0 + order by t1_distr.id +) s0 using id; + +select test_01103.t1_distr.id +from test_01103.t1_distr +join ( + select test_01103.t1_distr.id + from test_01103.t1_distr as d1 + inner join test_01103.t2_distr as d2 on test_01103.t1_distr.id = test_01103.t2_distr.id + where test_01103.t1_distr.id > 0 + order by test_01103.t1_distr.id +) s0 using id; + +DROP TABLE t1_shard; +DROP TABLE t2_shard; +DROP TABLE t1_distr; +DROP TABLE t2_distr; +DROP DATABASE test_01103; diff --git a/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.reference b/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.reference new file mode 100644 index 00000000000..c5079fa2cfd --- /dev/null +++ b/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.reference @@ -0,0 +1,4 @@ +100 +100 +100 +100 diff --git a/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.sql b/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.sql new file mode 100644 index 00000000000..b301c0ac00f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01104_distributed_numbers_test.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS d_numbers; +CREATE TABLE d_numbers (number UInt32) ENGINE = Distributed(test_cluster_two_shards_localhost, system, numbers, rand()); + +SET experimental_use_processors = 1; + +SELECT '100' AS number FROM d_numbers AS n WHERE n.number = 100 LIMIT 2; + +SET distributed_product_mode = 'local'; + +SELECT '100' AS number FROM d_numbers AS n WHERE n.number = 100 LIMIT 2; + +DROP TABLE d_numbers; diff --git a/dbms/tests/queries/0_stateless/01104_distributed_one_test.reference b/dbms/tests/queries/0_stateless/01104_distributed_one_test.reference new file mode 100644 index 00000000000..929dd64ae90 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01104_distributed_one_test.reference @@ -0,0 +1,6 @@ +local_0 1 +distributed_0 1 1 +distributed_0 2 1 +local_0 1 +distributed_0 1 1 +distributed_0 2 1 diff --git a/dbms/tests/queries/0_stateless/01104_distributed_one_test.sql b/dbms/tests/queries/0_stateless/01104_distributed_one_test.sql new file mode 100644 index 00000000000..92b4a83ebf3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01104_distributed_one_test.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS d_one; +CREATE TABLE d_one (dummy UInt8) ENGINE = Distributed(test_cluster_two_shards_localhost, system, one, rand()); + +SELECT 'local_0', toUInt8(1) AS dummy FROM system.one AS o WHERE o.dummy = 0; +SELECT 'local_1', toUInt8(1) AS dummy FROM system.one AS o WHERE o.dummy = 1; + +SELECT 'distributed_0', _shard_num, toUInt8(1) AS dummy FROM d_one AS o WHERE o.dummy = 0 ORDER BY _shard_num; +SELECT 'distributed_1', _shard_num, toUInt8(1) AS dummy FROM d_one AS o WHERE o.dummy = 1 ORDER BY _shard_num; + +SET distributed_product_mode = 'local'; + +SELECT 'local_0', toUInt8(1) AS dummy FROM system.one AS o WHERE o.dummy = 0; +SELECT 'local_1', toUInt8(1) AS dummy FROM system.one AS o WHERE o.dummy = 1; + +SELECT 'distributed_0', _shard_num, toUInt8(1) AS dummy FROM d_one AS o WHERE o.dummy = 0 ORDER BY _shard_num; +SELECT 'distributed_1', _shard_num, toUInt8(1) AS dummy FROM d_one AS o WHERE o.dummy = 1 ORDER BY _shard_num; + +DROP TABLE d_one; From 0fc7205ea365142a883019efae49c1961f391fe6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 28 Mar 2020 01:44:13 +0300 Subject: [PATCH 084/147] fix build after clang-tidy --- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/programs/copier/ClusterCopier.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 40f8bd2b99c..c4714ff201f 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -400,7 +400,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee bool ClusterCopier::checkPartitionPieceIsClean( const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock, - const String & task_status_path) const + const String & task_status_path) { LogicalClock task_start_clock; diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 12ac5bdf47e..4ad6265bb3c 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -88,10 +88,10 @@ protected: * The only requirement is that you have to pass is_dirty_flag_path and is_dirty_cleaned_path to the function. * And is_dirty_flag_path is a parent of is_dirty_cleaned_path. * */ - bool checkPartitionPieceIsClean( + static bool checkPartitionPieceIsClean( const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock, - const String & task_status_path) const; + const String & task_status_path); bool checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); @@ -114,7 +114,7 @@ protected: TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); /// Removes MATERIALIZED and ALIAS columns from create table query - ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); + static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); From 49865294d2fa7bbf2569ec0a0afd7e963aa41d78 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 04:15:09 +0300 Subject: [PATCH 085/147] Update RegexpRowInputFormat.cpp --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index f7afd5848c7..899630410a4 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -112,8 +112,9 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & size_t line_size = 0; - do { - char *pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end()); + do + { + char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end()); line_size += pos - buf.position(); buf.position() = pos; } while (buf.position() == buf.buffer().end() && !buf.eof()); From 8b87aea35e69671fbbaca1c8e24a0cf0d96ce88d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 04:17:49 +0300 Subject: [PATCH 086/147] Update RegexpRowInputFormat.h --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index de1520eba8e..8f9ecdc1349 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -36,7 +36,7 @@ public: private: bool readField(size_t index, MutableColumns & columns); void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); - ColumnFormat stringToFormat(const String & format); + static ColumnFormat stringToFormat(const String & format); PeekableReadBuffer buf; const FormatSettings format_settings; From 47c48111f3f840dbdd6f390b269021779c453e2d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 04:20:18 +0300 Subject: [PATCH 087/147] Update SystemLog.cpp --- dbms/src/Interpreters/SystemLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index 51a734b083f..a78342f8b17 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -58,7 +58,7 @@ std::shared_ptr createSystemLog( { String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)" - "SETTINGS index_granularity = 1024, min_bytes_for_wide_part = 10485760"; /// Use polymorphic parts for log tables by default + "SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default } size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); From c63ff92ec2783c07c661dfabdee84bc57ed46d57 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 04:30:21 +0300 Subject: [PATCH 088/147] Update IStorage.h --- dbms/src/Storages/IStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 5e6d6f1fa22..5387a7e0807 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -224,7 +224,7 @@ public: * SelectQueryInfo is required since the stage can depends on the query * (see Distributed() engine and optimize_skip_unused_shards). */ - QueryProcessingStage::Enum getQueryProcessingStage(const Context &context) const { return getQueryProcessingStage(context, {}); } + QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const { return getQueryProcessingStage(context, {}); } virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &, const ASTPtr &) const { return QueryProcessingStage::FetchColumns; } /** Watch live changes to the table. From 676aff9ba830875850a7574ff7f9fd459e712111 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 04:31:03 +0300 Subject: [PATCH 089/147] Update StorageDistributed.cpp --- dbms/src/Storages/StorageDistributed.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index afc730fbf9d..783d8318ab0 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -236,7 +236,7 @@ void replaceConstantExpressions(ASTPtr & node, const Context & context, const Na visitor.visit(node); } -} // \anonymous +} /// For destruction of std::unique_ptr of type that is incomplete in class definition. From 097fcbcadf69e39576bf906e2504e6ccc63a5b58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 04:53:53 +0300 Subject: [PATCH 090/147] Removed useless method --- dbms/src/Storages/IStorage.cpp | 8 -------- dbms/src/Storages/IStorage.h | 5 ----- 2 files changed, 13 deletions(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index ef303f78bd4..4d916ca1b46 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -336,14 +336,6 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_ return result; } -void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) -{ - if (!lock_holder.alter_intention_lock) - throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); - - lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); -} - void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { if (!lock_holder.alter_intention_lock) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 5387a7e0807..379d58a1843 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -205,11 +205,6 @@ public: /// can modify the table structure. It can later be upgraded to the exclusive lock. TableStructureWriteLockHolder lockAlterIntention(const String & query_id); - /// Upgrade alter intention lock and make sure that no new data is inserted into the table. - /// This is used by the ALTER MODIFY of the MergeTree storage to consistently determine - /// the set of parts that needs to be altered. - void lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id); - /// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries /// to ensure that no other query uses the table structure and it can be safely changed. void lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id); From f27a95da0d3b5d2dba8e8436a334ced0312797e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 04:59:06 +0300 Subject: [PATCH 091/147] Fixed typo --- dbms/src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6b32285b19a..85795f962bb 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -238,7 +238,7 @@ void StorageMergeTree::alter( /// We release all locks except alter_intention_lock which allows /// to execute alter queries sequentially - table_lock_holder.releaseAllExpectAlterIntention(); + table_lock_holder.releaseAllExceptAlterIntention(); /// Always execute required mutations synchronously, because alters /// should be executed in sequential order. From 1ea236c91d985f8737ad7df2bcd8c9173d784cf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 05:09:29 +0300 Subject: [PATCH 092/147] Remove useless code around locks --- .../PushingToViewsBlockOutputStream.cpp | 4 ++-- dbms/src/Functions/FunctionJoinGet.cpp | 2 +- .../Interpreters/InterpreterAlterQuery.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterDescribeQuery.cpp | 2 +- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 7 +----- dbms/src/Storages/IStorage.h | 7 +----- .../Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 4 ++-- dbms/src/Storages/StorageMerge.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.cpp | 18 +++++++------- .../Storages/StorageReplicatedMergeTree.cpp | 24 +++++++++---------- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- .../Storages/System/StorageSystemTables.cpp | 2 +- dbms/src/Storages/TableStructureLockHolder.h | 5 +--- 21 files changed, 43 insertions(+), 56 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 991d206777a..5752fbaff96 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -25,7 +25,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. */ - addTableLock(storage->lockStructureForShare(true, context.getInitialQueryId())); + addTableLock(storage->lockStructureForShare(context.getInitialQueryId())); /// If the "root" table deduplactes blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks @@ -54,7 +54,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { - addTableLock(materialized_view->lockStructureForShare(true, context.getInitialQueryId())); + addTableLock(materialized_view->lockStructureForShare(context.getInitialQueryId())); StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); diff --git a/dbms/src/Functions/FunctionJoinGet.cpp b/dbms/src/Functions/FunctionJoinGet.cpp index 0860deccb14..6a6c0c4a97e 100644 --- a/dbms/src/Functions/FunctionJoinGet.cpp +++ b/dbms/src/Functions/FunctionJoinGet.cpp @@ -65,7 +65,7 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName auto join = storage_join->getJoin(); DataTypes data_types(arguments.size()); - auto table_lock = storage_join->lockStructureForShare(false, context.getInitialQueryId()); + auto table_lock = storage_join->lockStructureForShare(context.getInitialQueryId()); for (size_t i = 0; i < arguments.size(); ++i) data_types[i] = arguments[i].type; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 315527765ef..3a6b2b6d42e 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -82,7 +82,7 @@ BlockIO InterpreterAlterQuery::execute() if (!mutation_commands.empty()) { - auto table_lock_holder = table->lockStructureForShare(false /* because mutation is executed asyncronously */, context.getCurrentQueryId()); + auto table_lock_holder = table->lockStructureForShare(context.getCurrentQueryId()); MutationsInterpreter(table, mutation_commands, context, false).validate(table_lock_holder); table->mutate(mutation_commands, context); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index f15796688e1..b57604828e1 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -403,7 +403,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}); /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. - as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId()); + as_storage_lock = as_storage->lockStructureForShare(context.getCurrentQueryId()); properties.columns = as_storage->getColumns(); /// Secondary indices make sense only for MergeTree family of storage engines. diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 1353c01ebf6..cf7bb0458e9 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -89,7 +89,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table = DatabaseCatalog::instance().getTable(table_id); } - auto table_lock = table->lockStructureForShare(false, context.getInitialQueryId()); + auto table_lock = table->lockStructureForShare(context.getInitialQueryId()); columns = table->getColumns(); } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index b4280ee20e6..f12ac68cede 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -109,7 +109,7 @@ BlockIO InterpreterInsertQuery::execute() BlockIO res; StoragePtr table = getTable(query); - auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId()); + auto table_lock = table->lockStructureForShare(context.getInitialQueryId()); auto query_sample_block = getSampleBlock(query, table); if (!query.table_function) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 6ba7da77511..3e4fb7d5909 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -255,7 +255,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - table_lock = storage->lockStructureForShare(false, context->getInitialQueryId()); + table_lock = storage->lockStructureForShare(context->getInitialQueryId()); table_id = storage->getStorageID(); } diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 4d916ca1b46..1ab7ba4a525 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -314,11 +314,9 @@ bool IStorage::isVirtualColumn(const String & column_name) const return getColumns().get(column_name).is_virtual; } -TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id) +TableStructureReadLockHolder IStorage::lockStructureForShare(const String & query_id) { TableStructureReadLockHolder result; - if (will_add_new_data) - result.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Read, query_id); result.structure_lock = structure_lock->getLock(RWLockImpl::Read, query_id); if (is_dropped) @@ -341,8 +339,6 @@ void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_hol if (!lock_holder.alter_intention_lock) throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (!lock_holder.new_data_structure_lock) - lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); lock_holder.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id); } @@ -354,7 +350,6 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); - result.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id); result.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id); return result; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 379d58a1843..24a2456fc8e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -199,7 +199,7 @@ public: /// Acquire this lock if you need the table structure to remain constant during the execution of /// the query. If will_add_new_data is true, this means that the query will add new data to the table /// (INSERT or a parts merge). - TableStructureReadLockHolder lockStructureForShare(bool will_add_new_data, const String & query_id); + TableStructureReadLockHolder lockStructureForShare(const String & query_id); /// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you /// can modify the table structure. It can later be upgraded to the exclusive lock. @@ -474,11 +474,6 @@ private: /// (e.g. SELECT, INSERT) can continue to execute. mutable RWLock alter_intention_lock = RWLockImpl::create(); - /// It is taken for share for the entire INSERT query and the entire merge of the parts (for MergeTree). - /// ALTER COLUMN queries acquire an exclusive lock to ensure that no new parts with the old structure - /// are added to the table and thus the set of parts to modify doesn't change. - mutable RWLock new_data_structure_lock = RWLockImpl::create(); - /// Lock for the table column structure (names, types, etc.) and data path. /// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP) /// and in share mode by other queries. diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index 6373c85a15d..1b216e8bec3 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -85,7 +85,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo try { - auto storage_lock = data.lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto storage_lock = data.lockStructureForShare(RWLockImpl::NO_QUERY); MergeTreeData::DataPartPtr part = findPart(part_name); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 77a5bca7a92..27ad6871573 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -57,7 +57,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() { /// TODO: Implement tryLockStructureForShare. - auto lock = storage.lockStructureForShare(false, ""); + auto lock = storage.lockStructureForShare(""); storage.clearOldTemporaryDirectories(); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 17b716d14c2..5c8f878503a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -203,7 +203,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na else if (part->name == part_name) { auto zookeeper = storage.getZooKeeper(); - auto table_lock = storage.lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto table_lock = storage.lockStructureForShare(RWLockImpl::NO_QUERY); auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4a4c92129bf..bbbe194d768 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -168,7 +168,7 @@ Pipes StorageBuffer::read( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - auto destination_lock = destination->lockStructureForShare(false, context.getCurrentQueryId()); + auto destination_lock = destination->lockStructureForShare(context.getCurrentQueryId()); const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [this, destination](const String& column_name) { diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index 3fb25bf8275..63031572cd6 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -185,7 +185,7 @@ Pipes StorageMaterializedView::read( const unsigned num_streams) { auto storage = getTargetTable(); - auto lock = storage->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock = storage->lockStructureForShare(context.getCurrentQueryId()); if (query_info.order_by_optimizer) query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); @@ -200,7 +200,7 @@ Pipes StorageMaterializedView::read( BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Context & context) { auto storage = getTargetTable(); - auto lock = storage->lockStructureForShare(true, context.getCurrentQueryId()); + auto lock = storage->lockStructureForShare(context.getCurrentQueryId()); auto stream = storage->write(query, context); stream->addTableLock(lock); return stream; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index f3322c7dfff..f102ee1c6f8 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -364,7 +364,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String { auto & table = iterator->table(); if (table.get() != this) - selected_tables.emplace_back(table, table->lockStructureForShare(false, query_id), iterator->name()); + selected_tables.emplace_back(table, table->lockStructureForShare(query_id), iterator->name()); iterator->next(); } @@ -389,7 +389,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { - selected_tables.emplace_back(storage, storage->lockStructureForShare(false, query_id), iterator->name()); + selected_tables.emplace_back(storage, storage->lockStructureForShare(query_id), iterator->name()); virtual_column->insert(iterator->name()); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 85795f962bb..a845dadef4c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -532,7 +532,7 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto table_lock_holder = lockStructureForShare(true, RWLockImpl::NO_QUERY); + auto table_lock_holder = lockStructureForShare(RWLockImpl::NO_QUERY); FutureMergedMutatedPart future_part; @@ -650,7 +650,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() bool StorageMergeTree::tryMutatePart() { - auto table_lock_holder = lockStructureForShare(true, RWLockImpl::NO_QUERY); + auto table_lock_holder = lockStructureForShare(RWLockImpl::NO_QUERY); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; FutureMergedMutatedPart future_part; @@ -775,7 +775,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { { /// TODO: Implement tryLockStructureForShare. - auto lock_structure = lockStructureForShare(false, ""); + auto lock_structure = lockStructureForShare(""); clearOldPartsFromFilesystem(); clearOldTemporaryDirectories(); } @@ -968,14 +968,14 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructureForShare(false, context.getCurrentQueryId()); + auto lock = lockStructureForShare(context.getCurrentQueryId()); freezePartition(command.partition, command.with_name, context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructureForShare(false, context.getCurrentQueryId()); + auto lock = lockStructureForShare(context.getCurrentQueryId()); freezeAll(command.with_name, context, lock); } break; @@ -1040,8 +1040,8 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); - auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(context.getCurrentQueryId()); + auto lock2 = source_table->lockStructureForShare(context.getCurrentQueryId()); Stopwatch watch; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table); @@ -1111,8 +1111,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); - auto lock2 = dest_table->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(context.getCurrentQueryId()); + auto lock2 = dest_table->lockStructureForShare(context.getCurrentQueryId()); auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 386e232cc60..6982b6ea5cc 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1025,7 +1025,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) ReservationPtr reserved_space = reserveSpacePreferringTTLRules(estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); - auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto table_lock = lockStructureForShare(RWLockImpl::NO_QUERY); FutureMergedMutatedPart future_merged_part(parts, entry.new_part_type); if (future_merged_part.name != entry.new_part_name) @@ -1160,7 +1160,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Can throw an exception. ReservationPtr reserved_space = reserveSpace(estimated_space_for_result, source_part->disk); - auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto table_lock = lockStructureForShare(RWLockImpl::NO_QUERY); MutableDataPartPtr new_part; Transaction transaction(*this); @@ -1514,7 +1514,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) PartDescriptions parts_to_add; DataPartsVector parts_to_remove; - auto table_lock_holder_dst_table = lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto table_lock_holder_dst_table = lockStructureForShare(RWLockImpl::NO_QUERY); for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i) { @@ -1576,7 +1576,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return 0; } - table_lock_holder_src_table = source_table->lockStructureForShare(false, RWLockImpl::NO_QUERY); + table_lock_holder_src_table = source_table->lockStructureForShare(RWLockImpl::NO_QUERY); DataPartStates valid_states{MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}; @@ -2699,7 +2699,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin TableStructureReadLockHolder table_lock_holder; if (!to_detached) - table_lock_holder = lockStructureForShare(true, RWLockImpl::NO_QUERY); + table_lock_holder = lockStructureForShare(RWLockImpl::NO_QUERY); /// Logging Stopwatch stopwatch; @@ -3415,14 +3415,14 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructureForShare(false, query_context.getCurrentQueryId()); + auto lock = lockStructureForShare(query_context.getCurrentQueryId()); freezePartition(command.partition, command.with_name, query_context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructureForShare(false, query_context.getCurrentQueryId()); + auto lock = lockStructureForShare(query_context.getCurrentQueryId()); freezeAll(command.with_name, query_context, lock); } break; @@ -4430,7 +4430,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { /// Critical section is not required (since grabOldParts() returns unique part set on each call) - auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY); + auto table_lock = lockStructureForShare(RWLockImpl::NO_QUERY); auto zookeeper = getZooKeeper(); DataPartsVector parts = grabOldParts(); @@ -4725,8 +4725,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ const Context & context) { /// First argument is true, because we possibly will add new data to current table. - auto lock1 = lockStructureForShare(true, context.getCurrentQueryId()); - auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(context.getCurrentQueryId()); + auto lock2 = source_table->lockStructureForShare(context.getCurrentQueryId()); Stopwatch watch; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table); @@ -4904,8 +4904,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); - auto lock2 = dest_table->lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(context.getCurrentQueryId()); + auto lock2 = dest_table->lockStructureForShare(context.getCurrentQueryId()); auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index cbf6ada9ed3..9af8904ab26 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -103,7 +103,7 @@ protected: try { - table_lock = storage->lockStructureForShare(false, query_id); + table_lock = storage->lockStructureForShare(query_id); } catch (const Exception & e) { diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index d8f564b0160..c212b30d268 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -192,7 +192,7 @@ StoragesInfo StoragesInfoStream::next() try { /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->lockStructureForShare(false, query_id); + info.table_lock = info.storage->lockStructureForShare(query_id); } catch (const Exception & e) { diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index bd19f7dd2a4..c57a0c107d6 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -217,7 +217,7 @@ protected: if (need_lock_structure) { table = tables_it->table(); - lock = table->lockStructureForShare(false, context.getCurrentQueryId()); + lock = table->lockStructureForShare(context.getCurrentQueryId()); } } catch (const Exception & e) diff --git a/dbms/src/Storages/TableStructureLockHolder.h b/dbms/src/Storages/TableStructureLockHolder.h index c97d187d524..6d77d312f92 100644 --- a/dbms/src/Storages/TableStructureLockHolder.h +++ b/dbms/src/Storages/TableStructureLockHolder.h @@ -15,9 +15,8 @@ struct TableStructureWriteLockHolder *this = TableStructureWriteLockHolder(); } - void releaseAllExpectAlterIntention() + void releaseAllExceptAlterIntention() { - new_data_structure_lock.reset(); structure_lock.reset(); } @@ -26,7 +25,6 @@ private: /// Order is important. RWLockImpl::LockHolder alter_intention_lock; - RWLockImpl::LockHolder new_data_structure_lock; RWLockImpl::LockHolder structure_lock; }; @@ -41,7 +39,6 @@ private: friend class IStorage; /// Order is important. - RWLockImpl::LockHolder new_data_structure_lock; RWLockImpl::LockHolder structure_lock; }; From 4ec30b01b1ade9902be9667fe2721908490be3fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 05:26:48 +0300 Subject: [PATCH 093/147] Remove some of the garbage from code --- dbms/src/Databases/DatabaseMySQL.cpp | 2 +- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 4 ++-- dbms/src/Storages/IStorage.cpp | 8 ++++---- dbms/src/Storages/IStorage.h | 4 ++-- dbms/src/Storages/LiveView/StorageLiveView.cpp | 4 ++-- dbms/src/Storages/LiveView/StorageLiveView.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- dbms/src/Storages/TableStructureLockHolder.h | 6 +++--- 9 files changed, 17 insertions(+), 17 deletions(-) diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index 959121585ea..ad40cff9e6b 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -358,7 +358,7 @@ void DatabaseMySQL::cleanOutdatedTables() ++iterator; else { - const auto table_lock = (*iterator)->lockAlterIntention(RWLockImpl::NO_QUERY); + const auto table_lock = (*iterator)->lockAlterIntention(); (*iterator)->shutdown(); (*iterator)->is_dropped = true; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 3a6b2b6d42e..ddf1e27af87 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -101,7 +101,7 @@ BlockIO InterpreterAlterQuery::execute() switch (command.type) { case LiveViewCommand::REFRESH: - live_view->refresh(context); + live_view->refresh(); break; } } @@ -109,7 +109,7 @@ BlockIO InterpreterAlterQuery::execute() if (!alter_commands.empty()) { - auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId()); + auto table_lock_holder = table->lockAlterIntention(); StorageInMemoryMetadata metadata = table->getInMemoryMetadata(); alter_commands.validate(metadata, context); alter_commands.prepare(metadata); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 1ab7ba4a525..c36a28b115f 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -324,10 +324,10 @@ TableStructureReadLockHolder IStorage::lockStructureForShare(const String & quer return result; } -TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_id) +TableStructureWriteLockHolder IStorage::lockAlterIntention() { TableStructureWriteLockHolder result; - result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id); + result.alter_lock = std::unique_lock(alter_lock); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); @@ -336,7 +336,7 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_ void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id) { - if (!lock_holder.alter_intention_lock) + if (!lock_holder.alter_lock) throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR); lock_holder.structure_lock = structure_lock->getLock(RWLockImpl::Write, query_id); @@ -345,7 +345,7 @@ void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_hol TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id) { TableStructureWriteLockHolder result; - result.alter_intention_lock = alter_intention_lock->getLock(RWLockImpl::Write, query_id); + result.alter_lock = std::unique_lock(alter_lock); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 24a2456fc8e..497d7bbe08d 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -203,7 +203,7 @@ public: /// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you /// can modify the table structure. It can later be upgraded to the exclusive lock. - TableStructureWriteLockHolder lockAlterIntention(const String & query_id); + TableStructureWriteLockHolder lockAlterIntention(); /// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries /// to ensure that no other query uses the table structure and it can be safely changed. @@ -472,7 +472,7 @@ private: /// If you hold this lock exclusively, you can be sure that no other structure modifying queries /// (e.g. ALTER, DROP) are concurrently executing. But queries that only read table structure /// (e.g. SELECT, INSERT) can continue to execute. - mutable RWLock alter_intention_lock = RWLockImpl::create(); + mutable std::mutex alter_lock; /// Lock for the table column structure (names, types, etc.) and data path. /// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 049110a3294..cea75bffbc8 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -517,9 +517,9 @@ void StorageLiveView::drop(TableStructureWriteLockHolder &) condition.notify_all(); } -void StorageLiveView::refresh(const Context & context) +void StorageLiveView::refresh() { - auto alter_lock = lockAlterIntention(context.getCurrentQueryId()); + auto alter_lock = lockAlterIntention(); { std::lock_guard lock(mutex); if (getNewBlocks()) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 9186132f99d..b3ed89f8d10 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -123,7 +123,7 @@ public: void startup() override; void shutdown() override; - void refresh(const Context & context); + void refresh(); Pipes read( const Names & column_names, diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index a845dadef4c..79079aa3095 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -236,7 +236,7 @@ void StorageMergeTree::alter( DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata); - /// We release all locks except alter_intention_lock which allows + /// We release all locks except alter_lock which allows /// to execute alter queries sequentially table_lock_holder.releaseAllExceptAlterIntention(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 6982b6ea5cc..3d4a3bd9451 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3210,7 +3210,7 @@ void StorageReplicatedMergeTree::alter( alter_entry.emplace(); mutation_znode.reset(); - /// We can safely read structure, because we guarded with alter_intention_lock + /// We can safely read structure, because we guarded with alter_lock if (is_readonly) throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); diff --git a/dbms/src/Storages/TableStructureLockHolder.h b/dbms/src/Storages/TableStructureLockHolder.h index 6d77d312f92..50f196517e3 100644 --- a/dbms/src/Storages/TableStructureLockHolder.h +++ b/dbms/src/Storages/TableStructureLockHolder.h @@ -12,7 +12,7 @@ struct TableStructureWriteLockHolder { void release() { - *this = TableStructureWriteLockHolder(); + *this = {}; } void releaseAllExceptAlterIntention() @@ -24,7 +24,7 @@ private: friend class IStorage; /// Order is important. - RWLockImpl::LockHolder alter_intention_lock; + std::unique_lock alter_lock; RWLockImpl::LockHolder structure_lock; }; @@ -32,7 +32,7 @@ struct TableStructureReadLockHolder { void release() { - *this = TableStructureReadLockHolder(); + *this = {}; } private: From 01ed894b65cb4d3bd684440548a225efa61e50c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 06:02:26 +0300 Subject: [PATCH 094/147] Whitespaces --- dbms/programs/local/LocalServer.cpp | 3 +-- dbms/src/Interpreters/executeQuery.cpp | 1 + dbms/src/Interpreters/executeQuery.h | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index ec2c01924f6..26752da5d87 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -148,7 +148,6 @@ try return Application::EXIT_OK; } - context = std::make_unique(Context::createGlobal()); context->makeGlobalContext(); context->setApplicationType(Context::ApplicationType::LOCAL); @@ -180,7 +179,7 @@ try setupUsers(); /// Limit on total number of concurrently executing queries. - /// There is no need for concurrent threads, override max_concurrent_queries. + /// There is no need for concurrent queries, override max_concurrent_queries. context->getProcessList().setMaxSize(0); /// Size of cache for uncompressed blocks. Zero means disabled. diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 4adbbf85e42..c9c66832f08 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -41,6 +41,7 @@ #include #include + namespace ProfileEvents { extern const Event QueryMaskingRulesMatch; diff --git a/dbms/src/Interpreters/executeQuery.h b/dbms/src/Interpreters/executeQuery.h index 64d055486aa..b6cb56b9e24 100644 --- a/dbms/src/Interpreters/executeQuery.h +++ b/dbms/src/Interpreters/executeQuery.h @@ -38,7 +38,7 @@ void executeQuery( /// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections) /// must be done separately. BlockIO executeQuery( - const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. + const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions... bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed. @@ -48,7 +48,7 @@ BlockIO executeQuery( QueryPipeline executeQueryWithProcessors( - const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. + const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions... bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList. QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed. From 219b39187d6fba90fa2c14ddd7f299f9c12fcf13 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 28 Mar 2020 12:40:06 +0300 Subject: [PATCH 095/147] Fix build --- dbms/src/Storages/TableStructureLockHolder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/TableStructureLockHolder.h b/dbms/src/Storages/TableStructureLockHolder.h index c97d187d524..b5fc0c620ad 100644 --- a/dbms/src/Storages/TableStructureLockHolder.h +++ b/dbms/src/Storages/TableStructureLockHolder.h @@ -15,7 +15,7 @@ struct TableStructureWriteLockHolder *this = TableStructureWriteLockHolder(); } - void releaseAllExpectAlterIntention() + void releaseAllExceptAlterIntention() { new_data_structure_lock.reset(); structure_lock.reset(); From 003671ee3907d37aa333da2a25bbd935fed1e059 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 10 Mar 2020 16:00:32 +0800 Subject: [PATCH 096/147] Cleanup arrayReduce.cpp --- dbms/src/Functions/array/arrayReduce.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/array/arrayReduce.cpp b/dbms/src/Functions/array/arrayReduce.cpp index e97607af135..3df07303eba 100644 --- a/dbms/src/Functions/array/arrayReduce.cpp +++ b/dbms/src/Functions/array/arrayReduce.cpp @@ -5,12 +5,10 @@ #include #include #include -#include #include #include #include #include -#include #include #include @@ -108,7 +106,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { - IAggregateFunction & agg_func = *aggregate_function.get(); + IAggregateFunction & agg_func = *aggregate_function; std::unique_ptr arena = std::make_unique(); /// Aggregate functions do not support constant columns. Therefore, we materialize them. @@ -132,7 +130,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum else if (const ColumnConst * const_arr = checkAndGetColumnConst(col)) { materialized_columns.emplace_back(const_arr->convertToFullColumn()); - const auto & materialized_arr = typeid_cast(*materialized_columns.back().get()); + const auto & materialized_arr = typeid_cast(*materialized_columns.back()); aggregate_arguments_vec[i] = &materialized_arr.getData(); offsets_i = &materialized_arr.getOffsets(); } From 6c4147501baeb709d3ae3ce45fa4eaed3716358f Mon Sep 17 00:00:00 2001 From: hcz Date: Sat, 28 Mar 2020 17:59:11 +0800 Subject: [PATCH 097/147] Register function arrayReduceInRanges --- dbms/src/Functions/array/registerFunctionsArray.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Functions/array/registerFunctionsArray.cpp b/dbms/src/Functions/array/registerFunctionsArray.cpp index 313ab7e27bb..084ed1ce702 100644 --- a/dbms/src/Functions/array/registerFunctionsArray.cpp +++ b/dbms/src/Functions/array/registerFunctionsArray.cpp @@ -33,6 +33,7 @@ void registerFunctionArrayFlatten(FunctionFactory &); void registerFunctionArrayWithConstant(FunctionFactory &); void registerFunctionArrayZip(FunctionFactory &); void registerFunctionArrayAUC(FunctionFactory &); +void registerFunctionArrayReduceInRanges(FunctionFactory &); void registerFunctionsArray(FunctionFactory & factory) { @@ -53,6 +54,7 @@ void registerFunctionsArray(FunctionFactory & factory) registerFunctionArraySlice(factory); registerFunctionArrayReverse(factory); registerFunctionArrayReduce(factory); + registerFunctionArrayReduceInRanges(factory); registerFunctionRange(factory); registerFunctionsEmptyArray(factory); registerFunctionEmptyArrayToSingle(factory); From 40207b072d3bcc64dee22edfd925b792016c736a Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 10 Mar 2020 16:01:00 +0800 Subject: [PATCH 098/147] Add function arrayReduceInRanges --- .../Functions/array/arrayReduceInRanges.cpp | 385 ++++++++++++++++++ 1 file changed, 385 insertions(+) create mode 100644 dbms/src/Functions/array/arrayReduceInRanges.cpp diff --git a/dbms/src/Functions/array/arrayReduceInRanges.cpp b/dbms/src/Functions/array/arrayReduceInRanges.cpp new file mode 100644 index 00000000000..a9f0e68bfa8 --- /dev/null +++ b/dbms/src/Functions/array/arrayReduceInRanges.cpp @@ -0,0 +1,385 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + + +/** Applies an aggregate function to value ranges in the array. + * The function does what arrayReduce do on a structure similar to segment tree. + * Space complexity: n * log(n) + * + * arrayReduceInRanges('agg', indices, lengths, arr1, ...) + */ +class FunctionArrayReduceInRanges : public IFunction +{ +public: + static const size_t minimum_step = 64; + static constexpr auto name = "arrayReduceInRanges"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + +private: + /// lazy initialization in getReturnTypeImpl + /// TODO: init in OverloadResolver + mutable AggregateFunctionPtr aggregate_function; +}; + + +DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + /// The first argument is a constant string with the name of the aggregate function + /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). + + if (arguments.size() < 4) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 4.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst(arguments[0].column.get()); + if (!aggregate_function_name_column) + throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * indices_type = checkAndGetDataType(arguments[1].type.get()); + if (!indices_type || !WhichDataType(*indices_type->getNestedType()).isNativeUInt()) + throw Exception("Second argument for function " + getName() + " must be array of ints.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * lengths_type = checkAndGetDataType(arguments[1].type.get()); + if (!lengths_type || !WhichDataType(*lengths_type->getNestedType()).isNativeUInt()) + throw Exception("Third argument for function " + getName() + " must be array of ints.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypes argument_types(arguments.size() - 3); + for (size_t i = 3, size = arguments.size(); i < size; ++i) + { + const DataTypeArray * arg = checkAndGetDataType(arguments[i].type.get()); + if (!arg) + throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type " + + arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + argument_types[i - 3] = arg->getNestedType(); + } + + if (!aggregate_function) + { + String aggregate_function_name_with_params = aggregate_function_name_column->getValue(); + + if (aggregate_function_name_with_params.empty()) + throw Exception("First argument for function " + getName() + " (name of aggregate function) cannot be empty.", + ErrorCodes::BAD_ARGUMENTS); + + String aggregate_function_name; + Array params_row; + getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params, + aggregate_function_name, params_row, "function " + getName()); + + aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row); + } + + return std::make_shared(aggregate_function->getReturnType()); +} + + +void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +{ + IAggregateFunction & agg_func = *aggregate_function; + std::unique_ptr arena = std::make_unique(); + + /// Aggregate functions do not support constant columns. Therefore, we materialize them. + std::vector materialized_columns; + + /// Handling indices + + const IColumn * indices_col = block.getByPosition(arguments[1]).column.get(); + + const IColumn * indices_data = nullptr; + const ColumnArray::Offsets * indices_offsets = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(indices_col)) + { + indices_data = &arr->getData(); + indices_offsets = &arr->getOffsets(); + } + else if (const ColumnConst * const_arr = checkAndGetColumnConst(indices_col)) + { + materialized_columns.emplace_back(const_arr->convertToFullColumn()); + const auto & materialized_arr = typeid_cast(*materialized_columns.back()); + indices_data = &materialized_arr.getData(); + indices_offsets = &materialized_arr.getOffsets(); + } + + /// Handling lengths + + const IColumn * lengths_col = block.getByPosition(arguments[2]).column.get(); + + const IColumn * lengths_data = nullptr; + const ColumnArray::Offsets * lengths_offsets = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(lengths_col)) + { + lengths_data = &arr->getData(); + lengths_offsets = &arr->getOffsets(); + } + else if (const ColumnConst * const_arr = checkAndGetColumnConst(indices_col)) + { + materialized_columns.emplace_back(const_arr->convertToFullColumn()); + const auto & materialized_arr = typeid_cast(*materialized_columns.back()); + lengths_data = &materialized_arr.getData(); + lengths_offsets = &materialized_arr.getOffsets(); + } + + if (*indices_offsets != *lengths_offsets) + throw Exception("Lengths of `indices` and `lengths` passed to " + getName() + " must be equal.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + + /// Handling arguments + /// The code is mostly copied from `arrayReduce`. Maybe create a utility header? + + const size_t num_arguments_columns = arguments.size() - 3; + + std::vector aggregate_arguments_vec(num_arguments_columns); + const ColumnArray::Offsets * offsets = nullptr; + + for (size_t i = 0; i < num_arguments_columns; ++i) + { + const IColumn * col = block.getByPosition(arguments[i + 3]).column.get(); + + const ColumnArray::Offsets * offsets_i = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(col)) + { + aggregate_arguments_vec[i] = &arr->getData(); + offsets_i = &arr->getOffsets(); + } + else if (const ColumnConst * const_arr = checkAndGetColumnConst(col)) + { + materialized_columns.emplace_back(const_arr->convertToFullColumn()); + const auto & materialized_arr = typeid_cast(*materialized_columns.back()); + aggregate_arguments_vec[i] = &materialized_arr.getData(); + offsets_i = &materialized_arr.getOffsets(); + } + else + throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + if (i == 0) + offsets = offsets_i; + else if (*offsets_i != *offsets) + throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + const IColumn ** aggregate_arguments = aggregate_arguments_vec.data(); + + /// Handling results + + MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn(); + ColumnArray * result_arr = static_cast(result_holder.get()); + IColumn & result_data = result_arr->getData(); + + result_arr->getOffsets().insert(indices_offsets->begin(), indices_offsets->end()); + + /// AggregateFunction's states should be inserted into column using specific way + auto res_col_aggregate_function = typeid_cast(&result_data); + + if (!res_col_aggregate_function && agg_func.isState()) + throw Exception("State function " + agg_func.getName() + " inserts results into non-state column " + + block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN); + + /// Perform the aggregation + + size_t begin = 0; + size_t end = 0; + size_t ranges_begin = 0; + size_t ranges_end = 0; + + for (size_t i = 0; i < input_rows_count; ++i) + { + begin = end; + end = (*offsets)[i]; + ranges_begin = ranges_end; + ranges_end = (*indices_offsets)[i]; + + /// We will allocate pre-aggregation places for each `minimun_place << level` rows. + /// The value of `level` starts from 0, and it will never exceed the number of bits in a `size_t`. + /// We calculate the offset (and thus size) of those places in each level. + size_t place_offsets[sizeof(size_t) * 8]; + size_t place_total = 0; + { + size_t place_in_level = (end - begin) / minimum_step; + + place_offsets[0] = place_in_level; + for (size_t level = 0; place_in_level; ++level) + { + place_in_level >>= 1; + place_total = place_offsets[level] + place_in_level; + place_offsets[level + 1] = place_total; + } + } + + PODArray places(place_total); + for (size_t j = 0; j < place_total; ++j) + { + places[j] = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData()); + try + { + agg_func.create(places[j]); + } + catch (...) + { + for (size_t k = 0; k < j; ++k) + agg_func.destroy(places[k]); + throw; + } + } + + SCOPE_EXIT({ + for (size_t j = 0; j < place_total; ++j) + agg_func.destroy(places[j]); + }); + + auto true_func = &agg_func; + /// Unnest consecutive trailing -State combinators + while (auto func = typeid_cast(true_func)) + true_func = func->getNestedFunction().get(); + + /// Pre-aggregate to the initial level + for (size_t j = 0; j < place_offsets[0]; ++j) + { + size_t local_begin = j * minimum_step; + size_t local_end = (j + 1) * minimum_step; + + for (size_t k = local_begin; k < local_end; ++k) + true_func->add(places[j], aggregate_arguments, begin + k, arena.get()); + } + + /// Pre-aggregate to the higher levels by merging + { + size_t place_in_level = place_offsets[0] >> 1; + size_t place_begin = 0; + + for (size_t level = 0; place_in_level; ++level) + { + size_t next_place_begin = place_offsets[level]; + + for (size_t j = 0; j < place_in_level; ++j) + { + true_func->merge(places[next_place_begin + j], places[place_begin + (j << 1)], arena.get()); + true_func->merge(places[next_place_begin + j], places[place_begin + (j << 1) + 1], arena.get()); + } + + place_in_level >>= 1; + place_begin = next_place_begin; + } + } + + for (size_t j = ranges_begin; j < ranges_end; ++j) + { + size_t local_begin = std::max(indices_data->getUInt(j) - 1, begin); + size_t local_end = std::min(local_begin + lengths_data->getUInt(j), end); + size_t place_begin = (local_begin + minimum_step - 1) / minimum_step; + size_t place_end = local_end / minimum_step; + + AggregateDataPtr place = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData()); + try + { + agg_func.create(place); + } + catch (...) + { + agg_func.destroy(place); + throw; + } + + SCOPE_EXIT({ + agg_func.destroy(place); + }); + + if (place_begin < place_end) + { + /// In this case, we can use pre-aggregated data. + + /// Aggregate rows before + for (size_t k = local_begin; k < place_begin * minimum_step; ++k) + true_func->add(place, aggregate_arguments, begin + k, arena.get()); + + /// Aggregate using pre-aggretated data + { + size_t level = 0; + size_t place_curr = place_begin; + + while (place_curr < place_end) + { + while (((place_curr >> level) & 1) == 0 && place_curr + (2 << level) <= place_end) + level += 1; + while (place_curr + (1 << level) > place_end) + level -= 1; + + size_t place_offset = 0; + if (level) + place_offset = place_offsets[level - 1]; + + true_func->merge(place, places[place_offset + (place_curr >> level)], arena.get()); + place_curr += 1 << level; + } + } + + /// Aggregate rows after + for (size_t k = place_end * minimum_step; k < local_end; ++k) + true_func->add(place, aggregate_arguments, begin + k, arena.get()); + } + else + { + /// In this case, we can not use pre-aggregated data. + + for (size_t k = local_begin; k < local_end; ++k) + true_func->add(place, aggregate_arguments, begin + k, arena.get()); + } + + if (!res_col_aggregate_function) + agg_func.insertResultInto(place, result_data); + else + res_col_aggregate_function->insertFrom(place); + } + } + + block.getByPosition(result).column = std::move(result_holder); +} + + +void registerFunctionArrayReduceInRanges(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} From 3e418f8af9717a6a85636d0e6b340a4799c22560 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 10 Mar 2020 17:21:55 +0800 Subject: [PATCH 099/147] Add tests for array_reduce_in_ranges --- .../01096_array_reduce_in_ranges.reference | 22 ++++++++++++++ .../01096_array_reduce_in_ranges.sql | 29 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference create mode 100644 dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql diff --git a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference new file mode 100644 index 00000000000..cd21e7adfca --- /dev/null +++ b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference @@ -0,0 +1,22 @@ +[['a','b','c'],['b','c','d'],['c','d','e']] +[0,0,0,0,0,0,0,100,200,300,400,0,0,300,500,700,400,0,0,600,900,700,400,0] +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql new file mode 100644 index 00000000000..1c90c2cfa2f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql @@ -0,0 +1,29 @@ +SELECT + arrayReduceInRanges( + 'groupArray', + [1, 2, 3], + [3, 3, 3], + ['a', 'b', 'c', 'd', 'e'] + ); + +SELECT + arrayReduceInRanges( + 'sum', + [0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5], + [0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3], + [100, 200, 300, 400] + ); + +WITH + arrayMap(x -> x + 1, range(50)) as data +SELECT + arrayReduceInRanges('groupArray', [a, b], [c, d], data) = + [arraySlice(data, a, c), arraySlice(data, b, d)] +FROM ( + SELECT + cityHash64(number + 100) % 40 as a, + cityHash64(number + 200) % 60 as b, + cityHash64(number + 300) % 20 as c, + cityHash64(number + 400) % 30 as d + FROM numbers(20) +); From d856313b15b6a57888448528db803b15d219b919 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 11 Mar 2020 10:42:38 +0800 Subject: [PATCH 100/147] Fix --- dbms/src/Functions/array/arrayReduceInRanges.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/array/arrayReduceInRanges.cpp b/dbms/src/Functions/array/arrayReduceInRanges.cpp index a9f0e68bfa8..4a2023d3745 100644 --- a/dbms/src/Functions/array/arrayReduceInRanges.cpp +++ b/dbms/src/Functions/array/arrayReduceInRanges.cpp @@ -153,7 +153,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers lengths_data = &arr->getData(); lengths_offsets = &arr->getOffsets(); } - else if (const ColumnConst * const_arr = checkAndGetColumnConst(indices_col)) + else if (const ColumnConst * const_arr = checkAndGetColumnConst(lengths_col)) { materialized_columns.emplace_back(const_arr->convertToFullColumn()); const auto & materialized_arr = typeid_cast(*materialized_columns.back()); @@ -305,8 +305,8 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers for (size_t j = ranges_begin; j < ranges_end; ++j) { - size_t local_begin = std::max(indices_data->getUInt(j) - 1, begin); - size_t local_end = std::min(local_begin + lengths_data->getUInt(j), end); + size_t local_begin = std::max(indices_data->getUInt(j) - 1, size_t(0)); + size_t local_end = std::min(local_begin + lengths_data->getUInt(j), end - begin); size_t place_begin = (local_begin + minimum_step - 1) / minimum_step; size_t place_end = local_end / minimum_step; From 421d45f04d3afdc10c07b6e30a3a46258b5f08ec Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 11 Mar 2020 10:43:26 +0800 Subject: [PATCH 101/147] Add performance tests for arrayReduce* functions --- dbms/tests/performance/array_reduce.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 dbms/tests/performance/array_reduce.xml diff --git a/dbms/tests/performance/array_reduce.xml b/dbms/tests/performance/array_reduce.xml new file mode 100644 index 00000000000..53f96550204 --- /dev/null +++ b/dbms/tests/performance/array_reduce.xml @@ -0,0 +1,16 @@ + + + + + 10000 + + + + + SELECT arrayReduce('count', range(100000000)) + SELECT arrayReduce('sum', range(100000000)) + SELECT arrayReduceInRanges('count', [1], [100000000], range(100000000)) + SELECT arrayReduceInRanges('sum', [1], [100000000], range(100000000)) + SELECT arrayReduceInRanges('count', range(1000000), range(1000000), range(100000000))[123456] + SELECT arrayReduceInRanges('sum', range(1000000), range(1000000), range(100000000))[123456] + From 294f4af16545ad6c2703ef03fa6e5af9811dab2d Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 11 Mar 2020 10:56:36 +0800 Subject: [PATCH 102/147] Add docs for arrayReduceInRanges --- .../functions/array_functions.md | 67 ++++++++++++++++--- 1 file changed, 56 insertions(+), 11 deletions(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 315b523054a..663b096fc3d 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -804,17 +804,30 @@ SELECT └──────────────┴───────────┘ ``` -## arrayReduce(agg\_func, arr1, …) {#array-functions-arrayreduce} +## arrayReduce {#arrayreduce} Applies an aggregate function to array elements and returns its result. The name of the aggregation function is passed as a string in single quotes `'max'`, `'sum'`. When using parametric aggregate functions, the parameter is indicated after the function name in parentheses `'uniqUpTo(6)'`. -Example: +**Syntax** -``` sql +```sql +arrayReduce(agg_func, arr1, arr2, ..., arrN) +``` + +**Parameters** + +* `agg_func` — The name of an aggregate function which should be a constant [string](../../data_types/string.md). +* `arr` — Any number of [array](../../data_types/array.md) type columns as the parameters of the aggregation function. + +**Returned value** + +**Example** + +```sql SELECT arrayReduce('max', [1, 2, 3]) ``` -``` text +```text ┌─arrayReduce('max', [1, 2, 3])─┐ │ 3 │ └───────────────────────────────┘ @@ -822,13 +835,11 @@ SELECT arrayReduce('max', [1, 2, 3]) If an aggregate function takes multiple arguments, then this function must be applied to multiple arrays of the same size. -Example: - -``` sql +```sql SELECT arrayReduce('maxIf', [3, 5], [1, 0]) ``` -``` text +```text ┌─arrayReduce('maxIf', [3, 5], [1, 0])─┐ │ 3 │ └──────────────────────────────────────┘ @@ -836,17 +847,51 @@ SELECT arrayReduce('maxIf', [3, 5], [1, 0]) Example with a parametric aggregate function: -``` sql +```sql SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) ``` -``` text +```text ┌─arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10])─┐ │ 4 │ └─────────────────────────────────────────────────────────────┘ ``` -## arrayReverse(arr) {#array_functions-arrayreverse} +## arrayReduceInRanges {#arrayreduceinranges} + +Applies an aggregate function to array elements in given ranges and returns an array containing the result corresponding to each range. The function will return the same result as multiple `arrayReduce(agg_func, arraySlice(arr1, index, length), ...)`. + +**Syntax** + +```sql +arrayReduceInRanges(agg_func, ranges, arr1, arr2, ..., arrN) +``` + +**Parameters** + +* `agg_func` — The name of an aggregate function which should be a constant [string](../../data_types/string.md). +* `ranges` — The ranges to aggretate which should be an [array](../../data_types/array.md) of [tuples](../../data_types/tuple.md) which containing the index and the length of each range. +* `arr` — Any number of [array](../../data_types/array.md) type columns as the parameters of the aggregation function. + +**Returned value** + +**Example** + +```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} Returns an array of the same size as the original array containing the elements in reverse order. From b634228947ed5e580dcbf750c81e09e806122834 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 11 Mar 2020 17:10:39 +0800 Subject: [PATCH 103/147] Use tuples in arrayReduceInRanges --- .../Functions/array/arrayReduceInRanges.cpp | 121 ++++++++++-------- dbms/tests/performance/array_reduce.xml | 8 +- .../01096_array_reduce_in_ranges.reference | 2 +- .../01096_array_reduce_in_ranges.sql | 13 +- 4 files changed, 82 insertions(+), 62 deletions(-) diff --git a/dbms/src/Functions/array/arrayReduceInRanges.cpp b/dbms/src/Functions/array/arrayReduceInRanges.cpp index 4a2023d3745..ddd2f95a3b3 100644 --- a/dbms/src/Functions/array/arrayReduceInRanges.cpp +++ b/dbms/src/Functions/array/arrayReduceInRanges.cpp @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -64,9 +66,9 @@ DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithType /// The first argument is a constant string with the name of the aggregate function /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). - if (arguments.size() < 4) + if (arguments.size() < 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be at least 4.", + + toString(arguments.size()) + ", should be at least 3.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst(arguments[0].column.get()); @@ -74,25 +76,30 @@ DataTypePtr FunctionArrayReduceInRanges::getReturnTypeImpl(const ColumnsWithType throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypeArray * indices_type = checkAndGetDataType(arguments[1].type.get()); - if (!indices_type || !WhichDataType(*indices_type->getNestedType()).isNativeUInt()) - throw Exception("Second argument for function " + getName() + " must be array of ints.", + const DataTypeArray * ranges_type_array = checkAndGetDataType(arguments[1].type.get()); + if (!ranges_type_array) + throw Exception("Second argument for function " + getName() + " must be an array of ranges.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + const DataTypeTuple * ranges_type_tuple = checkAndGetDataType(ranges_type_array->getNestedType().get()); + if (!ranges_type_tuple || ranges_type_tuple->getElements().size() != 2) + throw Exception("Each array element in the second argument for function " + getName() + " must be a tuple (index, length).", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isNativeInteger(ranges_type_tuple->getElements()[0])) + throw Exception("First tuple member in the second argument for function " + getName() + " must be ints or uints.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!WhichDataType(ranges_type_tuple->getElements()[1]).isNativeUInt()) + throw Exception("Second tuple member in the second argument for function " + getName() + " must be uints.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypeArray * lengths_type = checkAndGetDataType(arguments[1].type.get()); - if (!lengths_type || !WhichDataType(*lengths_type->getNestedType()).isNativeUInt()) - throw Exception("Third argument for function " + getName() + " must be array of ints.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - DataTypes argument_types(arguments.size() - 3); - for (size_t i = 3, size = arguments.size(); i < size; ++i) + DataTypes argument_types(arguments.size() - 2); + for (size_t i = 2, size = arguments.size(); i < size; ++i) { const DataTypeArray * arg = checkAndGetDataType(arguments[i].type.get()); if (!arg) throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type " + arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - argument_types[i - 3] = arg->getNestedType(); + argument_types[i - 2] = arg->getNestedType(); } if (!aggregate_function) @@ -123,59 +130,40 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers /// Aggregate functions do not support constant columns. Therefore, we materialize them. std::vector materialized_columns; - /// Handling indices + /// Handling ranges - const IColumn * indices_col = block.getByPosition(arguments[1]).column.get(); - - const IColumn * indices_data = nullptr; - const ColumnArray::Offsets * indices_offsets = nullptr; - if (const ColumnArray * arr = checkAndGetColumn(indices_col)) + const IColumn * ranges_col_array = block.getByPosition(arguments[1]).column.get(); + const IColumn * ranges_col_tuple = nullptr; + const ColumnArray::Offsets * ranges_offsets = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(ranges_col_array)) { - indices_data = &arr->getData(); - indices_offsets = &arr->getOffsets(); + ranges_col_tuple = &arr->getData(); + ranges_offsets = &arr->getOffsets(); } - else if (const ColumnConst * const_arr = checkAndGetColumnConst(indices_col)) + else if (const ColumnConst * const_arr = checkAndGetColumnConst(ranges_col_array)) { materialized_columns.emplace_back(const_arr->convertToFullColumn()); const auto & materialized_arr = typeid_cast(*materialized_columns.back()); - indices_data = &materialized_arr.getData(); - indices_offsets = &materialized_arr.getOffsets(); + ranges_col_tuple = &materialized_arr.getData(); + ranges_offsets = &materialized_arr.getOffsets(); } + else + throw Exception("Illegal column " + ranges_col_array->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - /// Handling lengths - - const IColumn * lengths_col = block.getByPosition(arguments[2]).column.get(); - - const IColumn * lengths_data = nullptr; - const ColumnArray::Offsets * lengths_offsets = nullptr; - if (const ColumnArray * arr = checkAndGetColumn(lengths_col)) - { - lengths_data = &arr->getData(); - lengths_offsets = &arr->getOffsets(); - } - else if (const ColumnConst * const_arr = checkAndGetColumnConst(lengths_col)) - { - materialized_columns.emplace_back(const_arr->convertToFullColumn()); - const auto & materialized_arr = typeid_cast(*materialized_columns.back()); - lengths_data = &materialized_arr.getData(); - lengths_offsets = &materialized_arr.getOffsets(); - } - - if (*indices_offsets != *lengths_offsets) - throw Exception("Lengths of `indices` and `lengths` passed to " + getName() + " must be equal.", - ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + const IColumn & indices_col = static_cast(ranges_col_tuple)->getColumn(0); + const IColumn & lengths_col = static_cast(ranges_col_tuple)->getColumn(1); /// Handling arguments /// The code is mostly copied from `arrayReduce`. Maybe create a utility header? - const size_t num_arguments_columns = arguments.size() - 3; + const size_t num_arguments_columns = arguments.size() - 2; std::vector aggregate_arguments_vec(num_arguments_columns); const ColumnArray::Offsets * offsets = nullptr; for (size_t i = 0; i < num_arguments_columns; ++i) { - const IColumn * col = block.getByPosition(arguments[i + 3]).column.get(); + const IColumn * col = block.getByPosition(arguments[i + 2]).column.get(); const ColumnArray::Offsets * offsets_i = nullptr; if (const ColumnArray * arr = checkAndGetColumn(col)) @@ -207,7 +195,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers ColumnArray * result_arr = static_cast(result_holder.get()); IColumn & result_data = result_arr->getData(); - result_arr->getOffsets().insert(indices_offsets->begin(), indices_offsets->end()); + result_arr->getOffsets().insert(ranges_offsets->begin(), ranges_offsets->end()); /// AggregateFunction's states should be inserted into column using specific way auto res_col_aggregate_function = typeid_cast(&result_data); @@ -228,7 +216,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers begin = end; end = (*offsets)[i]; ranges_begin = ranges_end; - ranges_end = (*indices_offsets)[i]; + ranges_end = (*ranges_offsets)[i]; /// We will allocate pre-aggregation places for each `minimun_place << level` rows. /// The value of `level` starts from 0, and it will never exceed the number of bits in a `size_t`. @@ -305,8 +293,37 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers for (size_t j = ranges_begin; j < ranges_end; ++j) { - size_t local_begin = std::max(indices_data->getUInt(j) - 1, size_t(0)); - size_t local_end = std::min(local_begin + lengths_data->getUInt(j), end - begin); + size_t local_begin = 0; + size_t local_end = 0; + + { + Int64 index = indices_col.getInt(j); + UInt64 length = lengths_col.getUInt(j); + + /// Keep the same as in arraySlice + + if (index > 0) + { + local_begin = index - 1; + if (local_begin + length < end - begin) + local_end = local_begin + length; + else + local_end = end - begin; + } + else if (index < 0) + { + if (end - begin + index > 0) + local_begin = end - begin + index; + else + local_begin = 0; + + if (local_begin + length < end - begin) + local_end = local_begin + length; + else + local_end = end - begin; + } + } + size_t place_begin = (local_begin + minimum_step - 1) / minimum_step; size_t place_end = local_end / minimum_step; diff --git a/dbms/tests/performance/array_reduce.xml b/dbms/tests/performance/array_reduce.xml index 53f96550204..7488199c149 100644 --- a/dbms/tests/performance/array_reduce.xml +++ b/dbms/tests/performance/array_reduce.xml @@ -9,8 +9,8 @@ SELECT arrayReduce('count', range(100000000)) SELECT arrayReduce('sum', range(100000000)) - SELECT arrayReduceInRanges('count', [1], [100000000], range(100000000)) - SELECT arrayReduceInRanges('sum', [1], [100000000], range(100000000)) - SELECT arrayReduceInRanges('count', range(1000000), range(1000000), range(100000000))[123456] - SELECT arrayReduceInRanges('sum', range(1000000), range(1000000), range(100000000))[123456] + SELECT arrayReduceInRanges('count', [(1, 100000000)], range(100000000)) + SELECT arrayReduceInRanges('sum', [(1, 100000000)], range(100000000)) + SELECT arrayReduceInRanges('count', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] + SELECT arrayReduceInRanges('sum', arrayZip(range(1000000), range(1000000)), range(100000000))[123456] diff --git a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference index cd21e7adfca..b74faffa6b3 100644 --- a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference +++ b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.reference @@ -1,5 +1,5 @@ [['a','b','c'],['b','c','d'],['c','d','e']] -[0,0,0,0,0,0,0,100,200,300,400,0,0,300,500,700,400,0,0,600,900,700,400,0] +[0,0,0,0,0,0,0,100,300,0,200,400,0,300,700,0,500,400,0,600,700,0,900,400] 1 1 1 diff --git a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql index 1c90c2cfa2f..051eaf7ca2f 100644 --- a/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql +++ b/dbms/tests/queries/0_stateless/01096_array_reduce_in_ranges.sql @@ -1,23 +1,26 @@ SELECT arrayReduceInRanges( 'groupArray', - [1, 2, 3], - [3, 3, 3], + [(1, 3), (2, 3), (3, 3)], ['a', 'b', 'c', 'd', 'e'] ); SELECT arrayReduceInRanges( 'sum', - [0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5, 0, 1, 2, 3, 4, 5], - [0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3], + [ + (-6, 0), (-4, 0), (-2, 0), (0, 0), (2, 0), (4, 0), + (-6, 1), (-4, 1), (-2, 1), (0, 1), (2, 1), (4, 1), + (-6, 2), (-4, 2), (-2, 2), (0, 2), (2, 2), (4, 2), + (-6, 3), (-4, 3), (-2, 3), (0, 3), (2, 3), (4, 3) + ], [100, 200, 300, 400] ); WITH arrayMap(x -> x + 1, range(50)) as data SELECT - arrayReduceInRanges('groupArray', [a, b], [c, d], data) = + arrayReduceInRanges('groupArray', [(a, c), (b, d)], data) = [arraySlice(data, a, c), arraySlice(data, b, d)] FROM ( SELECT From fbe951e03d6f34b70af4d3a7a0a0a542719e98ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 28 Mar 2020 13:26:23 +0300 Subject: [PATCH 104/147] Removed garbage --- dbms/src/Storages/LiveView/StorageLiveView.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index cea75bffbc8..93d183a594f 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -519,12 +519,9 @@ void StorageLiveView::drop(TableStructureWriteLockHolder &) void StorageLiveView::refresh() { - auto alter_lock = lockAlterIntention(); - { - std::lock_guard lock(mutex); - if (getNewBlocks()) - condition.notify_all(); - } + std::lock_guard lock(mutex); + if (getNewBlocks()) + condition.notify_all(); } Pipes StorageLiveView::read( From 3c98bfeeea401e26edc6dbd26db12bf1c41acaa1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 28 Mar 2020 15:01:55 +0300 Subject: [PATCH 105/147] Add correct version of llvm-symbolizer in stress-test --- docker/test/stress/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 116f4ec03f2..4d983ec4695 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -19,7 +19,7 @@ RUN apt-get update -y \ openssl \ netcat-openbsd \ telnet \ - llvm-8 \ + llvm-9 \ brotli COPY ./stress /stress @@ -33,7 +33,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/lib/llvm-8/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ + ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ service clickhouse-server start && sleep 5 \ From 20909252700adb21de4e6e8b1f14854d999590e9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 28 Mar 2020 11:11:49 +0300 Subject: [PATCH 106/147] Add type column into system.settings/merge_tree_settings This can be useful for various drivers to know the type. --- dbms/src/Core/SettingsCollection.h | 2 ++ dbms/src/Core/SettingsCollectionImpl.h | 4 ++- .../System/StorageSystemMergeTreeSettings.cpp | 2 ++ .../Storages/System/StorageSystemSettings.cpp | 4 ++- docs/en/operations/system_tables.md | 31 ++++++++++++++----- 5 files changed, 34 insertions(+), 9 deletions(-) diff --git a/dbms/src/Core/SettingsCollection.h b/dbms/src/Core/SettingsCollection.h index 466b80b1fd3..da21412b7c1 100644 --- a/dbms/src/Core/SettingsCollection.h +++ b/dbms/src/Core/SettingsCollection.h @@ -345,6 +345,7 @@ private: StringRef name; StringRef description; + StringRef type; bool is_important; IsChangedFunction is_changed; GetStringFunction get_string; @@ -391,6 +392,7 @@ public: const_reference(const const_reference & src) = default; const StringRef & getName() const { return member->name; } const StringRef & getDescription() const { return member->description; } + const StringRef & getType() const { return member->type; } bool isChanged() const { return member->is_changed(*collection); } Field getValue() const; String getValueAsString() const { return member->get_string(*collection); } diff --git a/dbms/src/Core/SettingsCollectionImpl.h b/dbms/src/Core/SettingsCollectionImpl.h index a97de0f48af..d5716c2a80d 100644 --- a/dbms/src/Core/SettingsCollectionImpl.h +++ b/dbms/src/Core/SettingsCollectionImpl.h @@ -323,7 +323,9 @@ void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFo #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ + add({StringRef(#NAME, strlen(#NAME)), \ + StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ + StringRef(#TYPE, strlen(#TYPE)), \ FLAGS & IMPORTANT, \ [](const Derived & d) { return d.NAME.changed; }, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \ diff --git a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.cpp index bb0918454ef..9d32ec74d51 100644 --- a/dbms/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/dbms/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -14,6 +14,7 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() {"value", std::make_shared()}, {"changed", std::make_shared()}, {"description", std::make_shared()}, + {"type", std::make_shared()}, }; } @@ -25,6 +26,7 @@ void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, const Conte res_columns[1]->insert(setting.getValueAsString()); res_columns[2]->insert(setting.isChanged()); res_columns[3]->insert(setting.getDescription().toString()); + res_columns[4]->insert(setting.getType().toString()); } } diff --git a/dbms/src/Storages/System/StorageSystemSettings.cpp b/dbms/src/Storages/System/StorageSystemSettings.cpp index 0be7618922d..f0a3e47defb 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.cpp +++ b/dbms/src/Storages/System/StorageSystemSettings.cpp @@ -17,7 +17,8 @@ NamesAndTypesList StorageSystemSettings::getNamesAndTypes() {"description", std::make_shared()}, {"min", std::make_shared(std::make_shared())}, {"max", std::make_shared(std::make_shared())}, - {"readonly", std::make_shared()} + {"readonly", std::make_shared()}, + {"type", std::make_shared()}, }; } @@ -59,6 +60,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, const Context res_columns[4]->insert(min); res_columns[5]->insert(max); res_columns[6]->insert(read_only); + res_columns[7]->insert(setting.getType().toString()); } } diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 4280c4c0ea7..c5613e92ae8 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -841,25 +841,42 @@ Columns: - `name` (String) — Setting name. - `value` (String) — Setting value. +- `description` (String) — Setting description. +- `type` (String) — Setting type. - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. +- `min` (Nullable(String)) — Get minimum allowed value (if any is set via [constraints](settings/constraints_on_settings.md#constraints-on-settings)). +- `max` (Nullable(String)) — Get maximum allowed value (if any is set via [constraints](settings/constraints_on_settings.md#constraints-on-settings)). +- `readonly` (UInt8) — Can user change this setting (for more info, look into [constraints](settings/constraints_on_settings.md#constraints-on-settings)). Example: ``` sql -SELECT * +SELECT name, value FROM system.settings WHERE changed ``` ``` text -┌─name───────────────────┬─value───────┬─changed─┐ -│ max_threads │ 8 │ 1 │ -│ use_uncompressed_cache │ 0 │ 1 │ -│ load_balancing │ random │ 1 │ -│ max_memory_usage │ 10000000000 │ 1 │ -└────────────────────────┴─────────────┴─────────┘ +┌─name───────────────────┬─value───────┐ +│ max_threads │ 8 │ +│ use_uncompressed_cache │ 0 │ +│ load_balancing │ random │ +│ max_memory_usage │ 10000000000 │ +└────────────────────────┴─────────────┘ ``` +## system.merge\_tree\_settings {#system-merge_tree_settings} + +Contains information about settings for `MergeTree` tables. + +Columns: + +- `name` (String) — Setting name. +- `value` (String) — Setting value. +- `description` (String) — Setting description. +- `type` (String) — Setting type. +- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. + ## system.table\_engines {#system-table-engines} Contains description of table engines supported by server and their feature support information. From 5cd9c764a9131bb9b170169582935bd0591b765f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 16:13:26 +0300 Subject: [PATCH 107/147] Update arrayReduceInRanges.cpp --- dbms/src/Functions/array/arrayReduceInRanges.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/array/arrayReduceInRanges.cpp b/dbms/src/Functions/array/arrayReduceInRanges.cpp index ddd2f95a3b3..79fb0e334ee 100644 --- a/dbms/src/Functions/array/arrayReduceInRanges.cpp +++ b/dbms/src/Functions/array/arrayReduceInRanges.cpp @@ -218,7 +218,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers ranges_begin = ranges_end; ranges_end = (*ranges_offsets)[i]; - /// We will allocate pre-aggregation places for each `minimun_place << level` rows. + /// We will allocate pre-aggregation places for each `minimum_place << level` rows. /// The value of `level` starts from 0, and it will never exceed the number of bits in a `size_t`. /// We calculate the offset (and thus size) of those places in each level. size_t place_offsets[sizeof(size_t) * 8]; From 654ed29f6c5040a44e7f9d3a8b1aa9a8fc4ec990 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 16:18:20 +0300 Subject: [PATCH 108/147] Update arrayReduceInRanges.cpp --- dbms/src/Functions/array/arrayReduceInRanges.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/dbms/src/Functions/array/arrayReduceInRanges.cpp b/dbms/src/Functions/array/arrayReduceInRanges.cpp index 79fb0e334ee..18f1086ea6a 100644 --- a/dbms/src/Functions/array/arrayReduceInRanges.cpp +++ b/dbms/src/Functions/array/arrayReduceInRanges.cpp @@ -328,15 +328,7 @@ void FunctionArrayReduceInRanges::executeImpl(Block & block, const ColumnNumbers size_t place_end = local_end / minimum_step; AggregateDataPtr place = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData()); - try - { - agg_func.create(place); - } - catch (...) - { - agg_func.destroy(place); - throw; - } + agg_func.create(place); SCOPE_EXIT({ agg_func.destroy(place); From 419150d81bef5e95262537ea5bba49c813002d99 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 28 Mar 2020 16:05:53 +0300 Subject: [PATCH 109/147] Basic system.settings/merge_tree_settings coverage --- .../01221_system_settings.reference | 4 +++ .../0_stateless/01221_system_settings.sql | 34 +++++++++++++++++++ 2 files changed, 38 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01221_system_settings.reference create mode 100644 dbms/tests/queries/0_stateless/01221_system_settings.sql diff --git a/dbms/tests/queries/0_stateless/01221_system_settings.reference b/dbms/tests/queries/0_stateless/01221_system_settings.reference new file mode 100644 index 00000000000..83c2c258c6e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01221_system_settings.reference @@ -0,0 +1,4 @@ +send_timeout 300 0 \N \N 0 SettingSeconds +replicated_max_parallel_sends 0 0 Limit parallel sends. SettingUInt64 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01221_system_settings.sql b/dbms/tests/queries/0_stateless/01221_system_settings.sql new file mode 100644 index 00000000000..5f4755add61 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01221_system_settings.sql @@ -0,0 +1,34 @@ +select * from system.settings where name = 'send_timeout'; +select * from system.merge_tree_settings order by length(description) limit 1; + +with [ + 'SettingSeconds', + 'SettingBool', + 'SettingInt64', + 'SettingString', + 'SettingChar', + 'SettingLogsLevel', + 'SettingURI', + 'SettingFloat', + 'SettingUInt64', + 'SettingMaxThreads', + 'SettingMilliseconds', + 'SettingJoinStrictness', + 'SettingJoinAlgorithm', + 'SettingOverflowMode', + 'SettingTotalsMode', + 'SettingLoadBalancing', + 'SettingOverflowModeGroupBy', + 'SettingDateTimeInputFormat', + 'SettingDistributedProductMode' +] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.settings; + +with [ + 'SettingSeconds', + 'SettingBool', + 'SettingInt64', + 'SettingString', + 'SettingFloat', + 'SettingUInt64', + 'SettingMaxThreads' +] as types select hasAll(arrayDistinct(groupArray(type)), types) from system.merge_tree_settings; From 546970fed2a784bbc4d90f557c96271adf92426a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 28 Mar 2020 20:02:59 +0300 Subject: [PATCH 110/147] Update system_tables.md --- docs/en/operations/system_tables.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index c5613e92ae8..4855dad0150 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -842,7 +842,7 @@ Columns: - `name` (String) — Setting name. - `value` (String) — Setting value. - `description` (String) — Setting description. -- `type` (String) — Setting type. +- `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. - `min` (Nullable(String)) — Get minimum allowed value (if any is set via [constraints](settings/constraints_on_settings.md#constraints-on-settings)). - `max` (Nullable(String)) — Get maximum allowed value (if any is set via [constraints](settings/constraints_on_settings.md#constraints-on-settings)). @@ -874,7 +874,7 @@ Columns: - `name` (String) — Setting name. - `value` (String) — Setting value. - `description` (String) — Setting description. -- `type` (String) — Setting type. +- `type` (String) — Setting type (implementation specific string value). - `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. ## system.table\_engines {#system-table-engines} From ab63bc5bf541893932cd928bce9cf62d14712765 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 29 Mar 2020 03:20:38 +0300 Subject: [PATCH 111/147] fix marks count for non-adaptive parts --- dbms/src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ac1df5c9ff8..79e4de3c9b0 100644 --- a/dbms/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -406,8 +406,8 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadColumns(require_columns_checksums); loadChecksums(require_columns_checksums); - calculateColumnsSizesOnDisk(); loadIndexGranularity(); + calculateColumnsSizesOnDisk(); loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` loadRowsCount(); /// Must be called after loadIndex() as it uses the value of `index_granularity`. loadPartitionAndMinMaxIndex(); From 93b7b0bd3aad6ff5150b98e7c2828901dde0c2ab Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 29 Mar 2020 18:07:51 +0800 Subject: [PATCH 112/147] Fix StorageJoin truncate reset overwrite flag --- dbms/src/Storages/StorageJoin.cpp | 5 ++-- dbms/src/Storages/StorageJoin.h | 1 + .../01230_join_get_truncate.reference | 3 +++ .../0_stateless/01230_join_get_truncate.sql | 27 +++++++++++++++++++ 4 files changed, 34 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01230_join_get_truncate.reference create mode 100644 dbms/tests/queries/0_stateless/01230_join_get_truncate.sql diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index f47cc9ae886..f5e88b193cd 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -43,7 +43,7 @@ StorageJoin::StorageJoin( ASTTableJoin::Strictness strictness_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool overwrite, + bool overwrite_, const Context & context_) : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_} , key_names(key_names_) @@ -51,6 +51,7 @@ StorageJoin::StorageJoin( , limits(limits_) , kind(kind_) , strictness(strictness_) + , overwrite(overwrite_) { for (const auto & key : key_names) if (!getColumns().hasPhysical(key)) @@ -69,7 +70,7 @@ void StorageJoin::truncate(const ASTPtr &, const Context &, TableStructureWriteL Poco::File(path + "tmp/").createDirectories(); increment = 0; - join = std::make_shared(table_join, getSampleBlock().sortColumns()); + join = std::make_shared(table_join, getSampleBlock().sortColumns(), overwrite); } diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index 6ee941feaec..acfc8a8b4e7 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -51,6 +51,7 @@ private: SizeLimits limits; ASTTableJoin::Kind kind; /// LEFT | INNER ... ASTTableJoin::Strictness strictness; /// ANY | ALL + bool overwrite; std::shared_ptr table_join; HashJoinPtr join; diff --git a/dbms/tests/queries/0_stateless/01230_join_get_truncate.reference b/dbms/tests/queries/0_stateless/01230_join_get_truncate.reference new file mode 100644 index 00000000000..d6dfd285646 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01230_join_get_truncate.reference @@ -0,0 +1,3 @@ +500 +1000 +1000 diff --git a/dbms/tests/queries/0_stateless/01230_join_get_truncate.sql b/dbms/tests/queries/0_stateless/01230_join_get_truncate.sql new file mode 100644 index 00000000000..b9c58fc5173 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01230_join_get_truncate.sql @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS join_test; + +CREATE TABLE join_test (id UInt16, num UInt16) engine = Join(ANY, LEFT, id) settings join_any_take_last_row = 1; + +INSERT INTO join_test (id, num) SELECT number, number FROM system.numbers LIMIT 1000; + +SELECT joinGet('join_test', 'num', 500); + +-- joinGet('join_test', 'num', 500) will be 500 and it is fine +-- replace all the values + +INSERT INTO join_test (id, num) SELECT number, number * 2 FROM system.numbers LIMIT 1000; + +SELECT joinGet ('join_test', 'num', 500); + +-- joinGet('join_test', 'num', 500) will be 1000 and it is fine + +TRUNCATE TABLE join_test; + +INSERT INTO join_test (id, num) SELECT number, number FROM system.numbers LIMIT 1000; + +INSERT INTO join_test (id, num) SELECT number, number * 2 FROM system.numbers LIMIT 1000; + +SELECT joinGet('join_test', 'num', 500); + +-- joinGet('join_test', 'num', 500) will be 1000 and it is not fine +DROP TABLE join_test; From 5def95f05b55bd1f636aa5a6e990648609d22c0c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 29 Mar 2020 16:21:26 +0300 Subject: [PATCH 113/147] Add test for HTTP GET provides readonly access. --- dbms/tests/integration/helpers/cluster.py | 21 +++++++++++++++++-- .../test_http_and_readonly/test.py | 20 ++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/integration/test_http_and_readonly/test.py diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 3537475ce96..b796c39151c 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -10,6 +10,7 @@ import socket import subprocess import time import urllib +import httplib import xml.dom.minidom import logging import docker @@ -652,7 +653,7 @@ class ClickHouseInstance: return self.client.query_and_get_answer_with_error(sql, stdin, timeout, settings, user, password) # Connects to the instance via HTTP interface, sends a query and returns the answer - def http_query(self, sql, data=None, params=None, user=None, password=None): + def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False): if params is None: params = {} else: @@ -668,7 +669,23 @@ class ClickHouseInstance: url = "http://" + auth + self.ip_address + ":8123/?" + urllib.urlencode(params) - return urllib.urlopen(url, data).read() + open_result = urllib.urlopen(url, data) + + def http_code_and_message(): + return str(open_result.getcode()) + " " + httplib.responses[open_result.getcode()] + ": " + open_result.read() + + if expect_fail_and_get_error: + if open_result.getcode() == 200: + raise Exception("ClickHouse HTTP server is expected to fail, but succeeded: " + open_result.read()) + return http_code_and_message() + else: + if open_result.getcode() != 200: + raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) + return open_result.read() + + # Connects to the instance via HTTP interface, sends a query, expects an error and return the error message + def http_query_and_get_error(self, sql, data=None, params=None, user=None, password=None): + return self.http_query(sql=sql, data=data, params=params, user=user, password=password, expect_fail_and_get_error=True) def kill_clickhouse(self, stop_start_wait_sec=5): pid = self.get_process_pid("clickhouse") diff --git a/dbms/tests/integration/test_http_and_readonly/test.py b/dbms/tests/integration/test_http_and_readonly/test.py new file mode 100644 index 00000000000..ea25b787a67 --- /dev/null +++ b/dbms/tests/integration/test_http_and_readonly/test.py @@ -0,0 +1,20 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') + + +@pytest.fixture(scope="module", autouse=True) +def setup_nodes(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_http_get_is_readonly(): + assert "Cannot execute query in readonly mode" in instance.http_query_and_get_error("CREATE TABLE xxx (a Date) ENGINE = MergeTree(a, a, 256)") + assert "Cannot modify 'readonly' setting in readonly mode" in instance.http_query_and_get_error("CREATE TABLE xxx (a Date) ENGINE = MergeTree(a, a, 256)", params={"readonly": 0}) From 0e9ddb9131f659066352b587ea085d82468c408b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 17:51:42 +0300 Subject: [PATCH 114/147] Add comments for columns of temporary tables in StorageSystemTables --- dbms/src/Storages/System/StorageSystemTables.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index c57a0c107d6..c4e6751f88b 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -141,51 +141,67 @@ protected: size_t src_index = 0; size_t res_index = 0; + // database if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // name if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.first); + // engine if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.second->getName()); + // is_temporary if (columns_mask[src_index++]) res_columns[res_index++]->insert(1u); + // data_paths if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // metadata_path if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // metadata_modification_time if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // dependencies_database if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // dependencies_table if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // create_table_query if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // engine_full if (columns_mask[src_index++]) res_columns[res_index++]->insert(table.second->getName()); + // partition_key if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // sorting_key if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // primary_key if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // sampling_key if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + // storage_policy if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); } From fbf72aee8db5618767f0413debd4e980df9a6ff7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 10:43:40 +0300 Subject: [PATCH 115/147] Install storage_policy for Distributed tables (follow-up for #8756) --- dbms/src/Storages/StorageDistributed.cpp | 7 +++++++ dbms/src/Storages/StorageDistributed.h | 1 + 2 files changed, 8 insertions(+) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 21c887d45cd..6f98d282e8c 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -530,6 +530,13 @@ namespace } +StoragePolicyPtr StorageDistributed::getStoragePolicy() const +{ + if (storage_policy.empty()) + return {}; + return global_context.getStoragePolicySelector()->get(storage_policy); +} + NameAndTypePair StorageDistributed::getColumn(const String & column_name) const { if (getColumns().hasPhysical(column_name)) diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 7f533d9588b..e12831709f7 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -60,6 +60,7 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } bool supportsPrewhere() const override { return true; } + StoragePolicyPtr getStoragePolicy() const override; NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; From 5c21dee05c559fe97b6c3b82ef3de5f6409512ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 29 Mar 2020 20:04:16 +0300 Subject: [PATCH 116/147] Every function in its own file, part 12 --- .../Functions/FunctionsMultiStringPosition.h | 130 ++++ .../Functions/FunctionsMultiStringSearch.h | 127 ++++ dbms/src/Functions/FunctionsStringRegex.cpp | 2 + dbms/src/Functions/FunctionsStringSearch.cpp | 707 ------------------ dbms/src/Functions/FunctionsStringSearch.h | 251 ------- .../Functions/FunctionsStringSearchToString.h | 89 +++ dbms/src/Functions/HasTokenImpl.h | 78 ++ .../Functions/MultiSearchAllPositionsImpl.h | 48 ++ .../src/Functions/MultiSearchFirstIndexImpl.h | 48 ++ .../Functions/MultiSearchFirstPositionImpl.h | 57 ++ dbms/src/Functions/MultiSearchImpl.h | 47 ++ dbms/src/Functions/PositionImpl.h | 307 ++++++++ dbms/src/Functions/URL/cutURLParameter.cpp | 2 +- .../src/Functions/URL/extractURLParameter.cpp | 2 +- dbms/src/Functions/hasToken.cpp | 22 + .../src/Functions/hasTokenCaseInsensitive.cpp | 25 + .../src/Functions/multiSearchAllPositions.cpp | 23 + ...multiSearchAllPositionsCaseInsensitive.cpp | 23 + ...iSearchAllPositionsCaseInsensitiveUTF8.cpp | 24 + .../Functions/multiSearchAllPositionsUTF8.cpp | 23 + dbms/src/Functions/multiSearchAny.cpp | 22 + .../multiSearchAnyCaseInsensitive.cpp | 22 + .../multiSearchAnyCaseInsensitiveUTF8.cpp | 23 + dbms/src/Functions/multiSearchAnyUTF8.cpp | 21 + dbms/src/Functions/multiSearchFirstIndex.cpp | 23 + .../multiSearchFirstIndexCaseInsensitive.cpp | 23 + ...ltiSearchFirstIndexCaseInsensitiveUTF8.cpp | 23 + .../Functions/multiSearchFirstIndexUTF8.cpp | 23 + .../Functions/multiSearchFirstPosition.cpp | 23 + ...ultiSearchFirstPositionCaseInsensitive.cpp | 23 + ...SearchFirstPositionCaseInsensitiveUTF8.cpp | 24 + .../multiSearchFirstPositionUTF8.cpp | 23 + dbms/src/Functions/position.cpp | 21 + .../src/Functions/positionCaseInsensitive.cpp | 20 + .../Functions/positionCaseInsensitiveUTF8.cpp | 22 + dbms/src/Functions/positionUTF8.cpp | 21 + .../registerFunctionsStringSearch.cpp | 66 ++ dbms/src/Functions/visitParamExtractRaw.cpp | 2 +- .../src/Functions/visitParamExtractString.cpp | 2 +- 39 files changed, 1500 insertions(+), 962 deletions(-) create mode 100644 dbms/src/Functions/FunctionsMultiStringPosition.h create mode 100644 dbms/src/Functions/FunctionsMultiStringSearch.h delete mode 100644 dbms/src/Functions/FunctionsStringSearch.cpp create mode 100644 dbms/src/Functions/FunctionsStringSearchToString.h create mode 100644 dbms/src/Functions/HasTokenImpl.h create mode 100644 dbms/src/Functions/MultiSearchAllPositionsImpl.h create mode 100644 dbms/src/Functions/MultiSearchFirstIndexImpl.h create mode 100644 dbms/src/Functions/MultiSearchFirstPositionImpl.h create mode 100644 dbms/src/Functions/MultiSearchImpl.h create mode 100644 dbms/src/Functions/PositionImpl.h create mode 100644 dbms/src/Functions/hasToken.cpp create mode 100644 dbms/src/Functions/hasTokenCaseInsensitive.cpp create mode 100644 dbms/src/Functions/multiSearchAllPositions.cpp create mode 100644 dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp create mode 100644 dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchAllPositionsUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchAny.cpp create mode 100644 dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp create mode 100644 dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchAnyUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchFirstIndex.cpp create mode 100644 dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp create mode 100644 dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchFirstIndexUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchFirstPosition.cpp create mode 100644 dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp create mode 100644 dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp create mode 100644 dbms/src/Functions/multiSearchFirstPositionUTF8.cpp create mode 100644 dbms/src/Functions/position.cpp create mode 100644 dbms/src/Functions/positionCaseInsensitive.cpp create mode 100644 dbms/src/Functions/positionCaseInsensitiveUTF8.cpp create mode 100644 dbms/src/Functions/positionUTF8.cpp create mode 100644 dbms/src/Functions/registerFunctionsStringSearch.cpp diff --git a/dbms/src/Functions/FunctionsMultiStringPosition.h b/dbms/src/Functions/FunctionsMultiStringPosition.h new file mode 100644 index 00000000000..9f98142ce1f --- /dev/null +++ b/dbms/src/Functions/FunctionsMultiStringPosition.h @@ -0,0 +1,130 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack + * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found + * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * + * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 + * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + + * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found + * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + + +template +class FunctionsMultiStringPosition : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); + if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(std::make_shared()); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + using ResultType = typename Impl::ResultType; + + const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; + + const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + + const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; + const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); + + if (!col_const_arr) + throw Exception( + "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", + ErrorCodes::ILLEGAL_COLUMN); + + Array src_arr = col_const_arr->getValue(); + + if (src_arr.size() > std::numeric_limits::max()) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + + ", should be at most 255", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector refs; + for (const auto & el : src_arr) + refs.emplace_back(el.get()); + + const size_t column_haystack_size = column_haystack->size(); + + auto col_res = ColumnVector::create(); + auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); + + auto & vec_res = col_res->getData(); + auto & offsets_res = col_offsets->getData(); + + vec_res.resize(column_haystack_size * refs.size()); + + if (col_haystack_vector) + Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); + + size_t refs_size = refs.size(); + size_t accum = refs_size; + + for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) + offsets_res[i] = accum; + + block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsMultiStringSearch.h b/dbms/src/Functions/FunctionsMultiStringSearch.h new file mode 100644 index 00000000000..ccaad0eaa9a --- /dev/null +++ b/dbms/src/Functions/FunctionsMultiStringSearch.h @@ -0,0 +1,127 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** + * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 + * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + + * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found + * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) + * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + + +/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. +/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions +template ::max()> +class FunctionsMultiStringSearch : public IFunction +{ + static_assert(LimitArgs > 0); + +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context & context) + { + if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) + throw Exception( + "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); + if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return Impl::getReturnType(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + using ResultType = typename Impl::ResultType; + + const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; + + const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + + const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; + const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); + + if (!col_const_arr) + throw Exception( + "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", + ErrorCodes::ILLEGAL_COLUMN); + + Array src_arr = col_const_arr->getValue(); + + if (src_arr.size() > LimitArgs) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) + + ", should be at most " + std::to_string(LimitArgs), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector refs; + refs.reserve(src_arr.size()); + + for (const auto & el : src_arr) + refs.emplace_back(el.get()); + + auto col_res = ColumnVector::create(); + auto col_offsets = ColumnArray::ColumnOffsets::create(); + + auto & vec_res = col_res->getData(); + auto & offsets_res = col_offsets->getData(); + + /// The blame for resizing output is for the callee. + if (col_haystack_vector) + Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res); + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); + + if constexpr (Impl::is_column_array) + block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); + else + block.getByPosition(result).column = std::move(col_res); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsStringRegex.cpp b/dbms/src/Functions/FunctionsStringRegex.cpp index 40a42fd2643..856852ae197 100644 --- a/dbms/src/Functions/FunctionsStringRegex.cpp +++ b/dbms/src/Functions/FunctionsStringRegex.cpp @@ -1,5 +1,7 @@ #include "FunctionsStringRegex.h" #include "FunctionsStringSearch.h" +#include "FunctionsMultiStringSearch.h" +#include "FunctionsStringSearchToString.h" #include #include #include diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp deleted file mode 100644 index 8279ded5f81..00000000000 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ /dev/null @@ -1,707 +0,0 @@ -#include "FunctionsStringSearch.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} -/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness. - */ -struct PositionCaseSensitiveASCII -{ - /// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization. - using SearcherInBigHaystack = Volnitsky; - - /// For search many substrings in one string - using MultiSearcherInBigHaystack = MultiVolnitsky; - - /// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization. - using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - /// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8). - static size_t countChars(const char * begin, const char * end) { return end - begin; } - - /// Convert string to lowercase. Only for case-insensitive search. - /// Implementation is permitted to be inefficient because it is called for single string. - static void toLowerIfNeed(std::string &) { } -}; - -struct PositionCaseInsensitiveASCII -{ - /// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it. - using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher; - using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive; - using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/) - { - return SearcherInBigHaystack(needle_data, needle_size); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) { return end - begin; } - - static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } -}; - -struct PositionCaseSensitiveUTF8 -{ - using SearcherInBigHaystack = VolnitskyUTF8; - using MultiSearcherInBigHaystack = MultiVolnitskyUTF8; - using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) - { - size_t res = 0; - for (auto it = begin; it != end; ++it) - if (!UTF8::isContinuationOctet(static_cast(*it))) - ++res; - return res; - } - - static void toLowerIfNeed(std::string &) { } -}; - -struct PositionCaseInsensitiveUTF8 -{ - using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8; - using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8; - using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal. - - static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) - { - return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); - } - - static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) - { - return SearcherInSmallHaystack(needle_data, needle_size); - } - - static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) - { - return MultiSearcherInBigHaystack(needles); - } - - static size_t countChars(const char * begin, const char * end) - { - size_t res = 0; - for (auto it = begin; it != end; ++it) - if (!UTF8::isContinuationOctet(static_cast(*it))) - ++res; - return res; - } - - static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); } -}; - -template -struct PositionImpl -{ - static constexpr bool use_default_implementation_for_constants = false; - - using ResultType = UInt64; - - /// Find one substring in many strings. - static void vectorConstant( - const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray & res) - { - const UInt8 * begin = data.data(); - const UInt8 * pos = begin; - const UInt8 * end = pos + data.size(); - - /// Current index in the array of strings. - size_t i = 0; - - typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos); - - /// We will search for the next occurrence in all strings at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Determine which index it refers to. - while (begin + offsets[i] <= pos) - { - res[i] = 0; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + needle.size() < begin + offsets[i]) - res[i] = 1 + Impl::countChars(reinterpret_cast(begin + offsets[i - 1]), reinterpret_cast(pos)); - else - res[i] = 0; - - pos = begin + offsets[i]; - ++i; - } - - if (i < res.size()) - memset(&res[i], 0, (res.size() - i) * sizeof(res[0])); - } - - /// Search for substring in string. - static void constantConstant(std::string data, std::string needle, UInt64 & res) - { - Impl::toLowerIfNeed(data); - Impl::toLowerIfNeed(needle); - - res = data.find(needle); - if (res == std::string::npos) - res = 0; - else - res = 1 + Impl::countChars(data.data(), data.data() + res); - } - - /// Search each time for a different single substring inside each time different string. - static void vectorVector( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - PaddedPODArray & res) - { - ColumnString::Offset prev_haystack_offset = 0; - ColumnString::Offset prev_needle_offset = 0; - - size_t size = haystack_offsets.size(); - - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - - if (0 == needle_size) - { - /// An empty string is always at the very beginning of `haystack`. - res[i] = 1; - } - else - { - /// It is assumed that the StringSearcher is not very difficult to initialize. - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( - reinterpret_cast(&needle_data[prev_needle_offset]), - needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end - - /// searcher returns a pointer to the found substring or to the end of `haystack`. - size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1]) - - &haystack_data[prev_haystack_offset]; - - if (pos != haystack_size) - { - res[i] = 1 - + Impl::countChars( - reinterpret_cast(&haystack_data[prev_haystack_offset]), - reinterpret_cast(&haystack_data[prev_haystack_offset + pos])); - } - else - res[i] = 0; - } - - prev_haystack_offset = haystack_offsets[i]; - prev_needle_offset = needle_offsets[i]; - } - } - - /// Find many substrings in single string. - static void constantVector( - const String & haystack, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - PaddedPODArray & res) - { - // NOTE You could use haystack indexing. But this is a rare case. - - ColumnString::Offset prev_needle_offset = 0; - - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - - if (0 == needle_size) - { - res[i] = 1; - } - else - { - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( - reinterpret_cast(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1); - - size_t pos = searcher.search( - reinterpret_cast(haystack.data()), - reinterpret_cast(haystack.data()) + haystack.size()) - - reinterpret_cast(haystack.data()); - - if (pos != haystack.size()) - { - res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos); - } - else - res[i] = 0; - } - - prev_needle_offset = needle_offsets[i]; - } - } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template -struct MultiSearchAllPositionsImpl -{ - using ResultType = UInt64; - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res) - { - auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 - { - return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); - }; - - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - - const size_t haystack_string_size = haystack_offsets.size(); - const size_t needles_size = needles.size(); - - /// Something can be uninitialized after the search itself - std::fill(res.begin(), res.end(), 0); - - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback); - prev_offset = haystack_offsets[j]; - } - } - } -}; - -template -struct MultiSearchImpl -{ - using ResultType = UInt8; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - if (iteration == 0 || !res[j]) - res[j] = searcher.searchOne(haystack, haystack_end); - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -template -struct MultiSearchFirstPositionImpl -{ - using ResultType = UInt64; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 - { - return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); - }; - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - if (iteration == 0 || res[j] == 0) - res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); - else - { - UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); - if (result != 0) - res[j] = std::min(result, res[j]); - } - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -template -struct MultiSearchFirstIndexImpl -{ - using ResultType = UInt64; - static constexpr bool is_using_hyperscan = false; - /// Variable for understanding, if we used offsets for the output, most - /// likely to determine whether the function returns ColumnVector of ColumnArray. - static constexpr bool is_column_array = false; - static auto getReturnType() { return std::make_shared>(); } - - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const std::vector & needles, - PaddedPODArray & res, - [[maybe_unused]] PaddedPODArray & offsets) - { - auto searcher = Impl::createMultiSearcherInBigHaystack(needles); - const size_t haystack_string_size = haystack_offsets.size(); - res.resize(haystack_string_size); - size_t iteration = 0; - while (searcher.hasMoreToSearch()) - { - size_t prev_offset = 0; - for (size_t j = 0; j < haystack_string_size; ++j) - { - const auto * haystack = &haystack_data[prev_offset]; - const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; - /// hasMoreToSearch traverse needles in increasing order - if (iteration == 0 || res[j] == 0) - res[j] = searcher.searchOneFirstIndex(haystack, haystack_end); - prev_offset = haystack_offsets[j]; - } - ++iteration; - } - } -}; - -/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. - */ -template -struct HasTokenImpl -{ - using ResultType = UInt8; - - static constexpr bool use_default_implementation_for_constants = true; - - static void vectorConstant( - const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) - { - if (offsets.empty()) - return; - - const UInt8 * begin = data.data(); - const UInt8 * pos = begin; - const UInt8 * end = pos + data.size(); - - /// The current index in the array of strings. - size_t i = 0; - - TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (begin + offsets[i] <= pos) - { - res[i] = negate_result; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + pattern.size() < begin + offsets[i]) - res[i] = !negate_result; - else - res[i] = negate_result; - - pos = begin + offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); - } - - template - static void vectorVector(Args &&...) - { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); - } - - /// Search different needles in single haystack. - template - static void constantVector(Args &&...) - { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); - } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -struct NamePosition -{ - static constexpr auto name = "position"; -}; -struct NamePositionUTF8 -{ - static constexpr auto name = "positionUTF8"; -}; -struct NamePositionCaseInsensitive -{ - static constexpr auto name = "positionCaseInsensitive"; -}; -struct NamePositionCaseInsensitiveUTF8 -{ - static constexpr auto name = "positionCaseInsensitiveUTF8"; -}; -struct NameMultiSearchAllPositions -{ - static constexpr auto name = "multiSearchAllPositions"; -}; -struct NameMultiSearchAllPositionsUTF8 -{ - static constexpr auto name = "multiSearchAllPositionsUTF8"; -}; -struct NameMultiSearchAllPositionsCaseInsensitive -{ - static constexpr auto name = "multiSearchAllPositionsCaseInsensitive"; -}; -struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8"; -}; -struct NameMultiSearchAny -{ - static constexpr auto name = "multiSearchAny"; -}; -struct NameMultiSearchAnyUTF8 -{ - static constexpr auto name = "multiSearchAnyUTF8"; -}; -struct NameMultiSearchAnyCaseInsensitive -{ - static constexpr auto name = "multiSearchAnyCaseInsensitive"; -}; -struct NameMultiSearchAnyCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8"; -}; -struct NameMultiSearchFirstIndex -{ - static constexpr auto name = "multiSearchFirstIndex"; -}; -struct NameMultiSearchFirstIndexUTF8 -{ - static constexpr auto name = "multiSearchFirstIndexUTF8"; -}; -struct NameMultiSearchFirstIndexCaseInsensitive -{ - static constexpr auto name = "multiSearchFirstIndexCaseInsensitive"; -}; -struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8"; -}; -struct NameMultiSearchFirstPosition -{ - static constexpr auto name = "multiSearchFirstPosition"; -}; -struct NameMultiSearchFirstPositionUTF8 -{ - static constexpr auto name = "multiSearchFirstPositionUTF8"; -}; -struct NameMultiSearchFirstPositionCaseInsensitive -{ - static constexpr auto name = "multiSearchFirstPositionCaseInsensitive"; -}; -struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 -{ - static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; -}; - -struct NameHasToken -{ - static constexpr auto name = "hasToken"; -}; - -struct NameHasTokenCaseInsensitive -{ - static constexpr auto name = "hasTokenCaseInsensitive"; -}; - - -using FunctionPosition = FunctionsStringSearch, NamePosition>; -using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; -using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; -using FunctionPositionCaseInsensitiveUTF8 - = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; - -using FunctionMultiSearchAllPositions - = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; -using FunctionMultiSearchAllPositionsUTF8 - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; -using FunctionMultiSearchAllPositionsCaseInsensitive - = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; -using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition< - MultiSearchAllPositionsImpl, - NameMultiSearchAllPositionsCaseInsensitiveUTF8>; - -using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; -using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; -using FunctionMultiSearchCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; -using FunctionMultiSearchCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; - -using FunctionMultiSearchFirstIndex - = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; -using FunctionMultiSearchFirstIndexUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; -using FunctionMultiSearchFirstIndexCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; -using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; - -using FunctionMultiSearchFirstPosition - = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; -using FunctionMultiSearchFirstPositionUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; -using FunctionMultiSearchFirstPositionCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; -using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch< - MultiSearchFirstPositionImpl, - NameMultiSearchFirstPositionCaseInsensitiveUTF8>; - -using FunctionHasToken = FunctionsStringSearch, NameHasToken>; -using FunctionHasTokenCaseInsensitive - = FunctionsStringSearch, NameHasTokenCaseInsensitive>; - -void registerFunctionsStringSearch(FunctionFactory & factory) -{ - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - - factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); -} -} diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index 2ae5f3598a8..9f4360cf935 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -10,9 +9,7 @@ #include #include #include -#include #include -#include namespace DB @@ -37,40 +34,12 @@ namespace DB * - the zero subpattern (the match part, otherwise); * - if not match - an empty string. * extract(haystack, pattern) - * - * replaceOne(haystack, pattern, replacement) - replacing the pattern with the specified rules, only the first occurrence. - * replaceAll(haystack, pattern, replacement) - replacing the pattern with the specified rules, all occurrences. - * - * replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence. - * replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences. - * - * multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack - * multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found - * multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * - * multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1 - * multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - - * multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found - * multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) - * multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) */ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int FUNCTION_NOT_ALLOWED; } template @@ -163,224 +132,4 @@ public: } }; - -template -class FunctionsStringSearchToString : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isString(arguments[1])) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; - - const ColumnConst * col_needle = typeid_cast(&*column_needle); - if (!col_needle) - throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN); - - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - - ColumnString::Chars & vec_res = col_res->getChars(); - ColumnString::Offsets & offsets_res = col_res->getOffsets(); - Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); - - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -template -class FunctionsMultiStringPosition : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(std::make_shared()); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - using ResultType = typename Impl::ResultType; - - const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; - - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - - const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - - if (!col_const_arr) - throw Exception( - "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", - ErrorCodes::ILLEGAL_COLUMN); - - Array src_arr = col_const_arr->getValue(); - - if (src_arr.size() > std::numeric_limits::max()) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) - + ", should be at most 255", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - std::vector refs; - for (const auto & el : src_arr) - refs.emplace_back(el.get()); - - const size_t column_haystack_size = column_haystack->size(); - - auto col_res = ColumnVector::create(); - auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size); - - auto & vec_res = col_res->getData(); - auto & offsets_res = col_offsets->getData(); - - vec_res.resize(column_haystack_size * refs.size()); - - if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res); - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); - - size_t refs_size = refs.size(); - size_t accum = refs_size; - - for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size) - offsets_res[i] = accum; - - block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); - } -}; - -/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. -/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions -template ::max()> -class FunctionsMultiStringSearch : public IFunction -{ - static_assert(LimitArgs > 0); - -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context & context) - { - if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan) - throw Exception( - "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const DataTypeArray * array_type = checkAndGetDataType(arguments[1].get()); - if (!array_type || !checkAndGetDataType(array_type->getNestedType().get())) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return Impl::getReturnType(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - using ResultType = typename Impl::ResultType; - - const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column; - - const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); - - const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column; - const ColumnConst * col_const_arr = checkAndGetColumnConst(arr_ptr.get()); - - if (!col_const_arr) - throw Exception( - "Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const", - ErrorCodes::ILLEGAL_COLUMN); - - Array src_arr = col_const_arr->getValue(); - - if (src_arr.size() > LimitArgs) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size()) - + ", should be at most " + std::to_string(LimitArgs), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - std::vector refs; - refs.reserve(src_arr.size()); - - for (const auto & el : src_arr) - refs.emplace_back(el.get()); - - auto col_res = ColumnVector::create(); - auto col_offsets = ColumnArray::ColumnOffsets::create(); - - auto & vec_res = col_res->getData(); - auto & offsets_res = col_offsets->getData(); - - /// The blame for resizing output is for the callee. - if (col_haystack_vector) - Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res); - else - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN); - - if constexpr (Impl::is_column_array) - block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets)); - else - block.getByPosition(result).column = std::move(col_res); - } -}; - } diff --git a/dbms/src/Functions/FunctionsStringSearchToString.h b/dbms/src/Functions/FunctionsStringSearchToString.h new file mode 100644 index 00000000000..91d3a21a823 --- /dev/null +++ b/dbms/src/Functions/FunctionsStringSearchToString.h @@ -0,0 +1,89 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/** Applies regexp re2 and extracts: + * - the first subpattern, if the regexp has a subpattern; + * - the zero subpattern (the match part, otherwise); + * - if not match - an empty string. + * extract(haystack, pattern) + */ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int FUNCTION_NOT_ALLOWED; +} + + +template +class FunctionsStringSearchToString : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isString(arguments[1])) + throw Exception( + "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; + + const ColumnConst * col_needle = typeid_cast(&*column_needle); + if (!col_needle) + throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN); + + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + + ColumnString::Chars & vec_res = col_res->getChars(); + ColumnString::Offsets & offsets_res = col_res->getOffsets(); + Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); + + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/HasTokenImpl.h b/dbms/src/Functions/HasTokenImpl.h new file mode 100644 index 00000000000..9adadee83a4 --- /dev/null +++ b/dbms/src/Functions/HasTokenImpl.h @@ -0,0 +1,78 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. + */ +template +struct HasTokenImpl +{ + using ResultType = UInt8; + + static constexpr bool use_default_implementation_for_constants = true; + + static void vectorConstant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) + { + if (offsets.empty()) + return; + + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// The current index in the array of strings. + size_t i = 0; + + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = negate_result; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + offsets[i]) + res[i] = !negate_result; + else + res[i] = negate_result; + + pos = begin + offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0])); + } + + template + static void vectorVector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + /// Search different needles in single haystack. + template + static void constantVector(Args &&...) + { + throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + } + + template + static void vectorFixedConstant(Args &&...) + { + throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchAllPositionsImpl.h b/dbms/src/Functions/MultiSearchAllPositionsImpl.h new file mode 100644 index 00000000000..f54fe41f20c --- /dev/null +++ b/dbms/src/Functions/MultiSearchAllPositionsImpl.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchAllPositionsImpl +{ + using ResultType = UInt64; + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res) + { + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + + const size_t haystack_string_size = haystack_offsets.size(); + const size_t needles_size = needles.size(); + + /// Something can be uninitialized after the search itself + std::fill(res.begin(), res.end(), 0); + + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback); + prev_offset = haystack_offsets[j]; + } + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchFirstIndexImpl.h b/dbms/src/Functions/MultiSearchFirstIndexImpl.h new file mode 100644 index 00000000000..bb0bb57b9db --- /dev/null +++ b/dbms/src/Functions/MultiSearchFirstIndexImpl.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchFirstIndexImpl +{ + using ResultType = UInt64; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + /// hasMoreToSearch traverse needles in increasing order + if (iteration == 0 || res[j] == 0) + res[j] = searcher.searchOneFirstIndex(haystack, haystack_end); + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchFirstPositionImpl.h b/dbms/src/Functions/MultiSearchFirstPositionImpl.h new file mode 100644 index 00000000000..4743518e61e --- /dev/null +++ b/dbms/src/Functions/MultiSearchFirstPositionImpl.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchFirstPositionImpl +{ + using ResultType = UInt64; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64 + { + return 1 + Impl::countChars(reinterpret_cast(start), reinterpret_cast(end)); + }; + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + if (iteration == 0 || res[j] == 0) + res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + else + { + UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback); + if (result != 0) + res[j] = std::min(result, res[j]); + } + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/MultiSearchImpl.h b/dbms/src/Functions/MultiSearchImpl.h new file mode 100644 index 00000000000..5b881e2fd55 --- /dev/null +++ b/dbms/src/Functions/MultiSearchImpl.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct MultiSearchImpl +{ + using ResultType = UInt8; + static constexpr bool is_using_hyperscan = false; + /// Variable for understanding, if we used offsets for the output, most + /// likely to determine whether the function returns ColumnVector of ColumnArray. + static constexpr bool is_column_array = false; + static auto getReturnType() { return std::make_shared>(); } + + static void vectorConstant( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const std::vector & needles, + PaddedPODArray & res, + [[maybe_unused]] PaddedPODArray & offsets) + { + auto searcher = Impl::createMultiSearcherInBigHaystack(needles); + const size_t haystack_string_size = haystack_offsets.size(); + res.resize(haystack_string_size); + size_t iteration = 0; + while (searcher.hasMoreToSearch()) + { + size_t prev_offset = 0; + for (size_t j = 0; j < haystack_string_size; ++j) + { + const auto * haystack = &haystack_data[prev_offset]; + const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1; + if (iteration == 0 || !res[j]) + res[j] = searcher.searchOne(haystack, haystack_end); + prev_offset = haystack_offsets[j]; + } + ++iteration; + } + } +}; + +} diff --git a/dbms/src/Functions/PositionImpl.h b/dbms/src/Functions/PositionImpl.h new file mode 100644 index 00000000000..bc0b2c7bcfb --- /dev/null +++ b/dbms/src/Functions/PositionImpl.h @@ -0,0 +1,307 @@ +#include "FunctionsStringSearch.h" + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness. + */ +struct PositionCaseSensitiveASCII +{ + /// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization. + using SearcherInBigHaystack = Volnitsky; + + /// For search many substrings in one string + using MultiSearcherInBigHaystack = MultiVolnitsky; + + /// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization. + using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + /// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8). + static size_t countChars(const char * begin, const char * end) { return end - begin; } + + /// Convert string to lowercase. Only for case-insensitive search. + /// Implementation is permitted to be inefficient because it is called for single string. + static void toLowerIfNeed(std::string &) { } +}; + + +struct PositionCaseInsensitiveASCII +{ + /// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it. + using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher; + using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive; + using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/) + { + return SearcherInBigHaystack(needle_data, needle_size); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) { return end - begin; } + + static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + + +struct PositionCaseSensitiveUTF8 +{ + using SearcherInBigHaystack = VolnitskyUTF8; + using MultiSearcherInBigHaystack = MultiVolnitskyUTF8; + using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher; + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) + { + size_t res = 0; + for (auto it = begin; it != end; ++it) + if (!UTF8::isContinuationOctet(static_cast(*it))) + ++res; + return res; + } + + static void toLowerIfNeed(std::string &) { } +}; + + +struct PositionCaseInsensitiveUTF8 +{ + using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8; + using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8; + using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal. + + static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint) + { + return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint); + } + + static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size) + { + return SearcherInSmallHaystack(needle_data, needle_size); + } + + static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector & needles) + { + return MultiSearcherInBigHaystack(needles); + } + + static size_t countChars(const char * begin, const char * end) + { + size_t res = 0; + for (auto it = begin; it != end; ++it) + if (!UTF8::isContinuationOctet(static_cast(*it))) + ++res; + return res; + } + + static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); } +}; + + +template +struct PositionImpl +{ + static constexpr bool use_default_implementation_for_constants = false; + + using ResultType = UInt64; + + /// Find one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray & res) + { + const UInt8 * begin = data.data(); + const UInt8 * pos = begin; + const UInt8 * end = pos + data.size(); + + /// Current index in the array of strings. + size_t i = 0; + + typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos); + + /// We will search for the next occurrence in all strings at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Determine which index it refers to. + while (begin + offsets[i] <= pos) + { + res[i] = 0; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + needle.size() < begin + offsets[i]) + res[i] = 1 + Impl::countChars(reinterpret_cast(begin + offsets[i - 1]), reinterpret_cast(pos)); + else + res[i] = 0; + + pos = begin + offsets[i]; + ++i; + } + + if (i < res.size()) + memset(&res[i], 0, (res.size() - i) * sizeof(res[0])); + } + + /// Search for substring in string. + static void constantConstant(std::string data, std::string needle, UInt64 & res) + { + Impl::toLowerIfNeed(data); + Impl::toLowerIfNeed(needle); + + res = data.find(needle); + if (res == std::string::npos) + res = 0; + else + res = 1 + Impl::countChars(data.data(), data.data() + res); + } + + /// Search each time for a different single substring inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + PaddedPODArray & res) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + if (0 == needle_size) + { + /// An empty string is always at the very beginning of `haystack`. + res[i] = 1; + } + else + { + /// It is assumed that the StringSearcher is not very difficult to initialize. + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( + reinterpret_cast(&needle_data[prev_needle_offset]), + needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end + + /// searcher returns a pointer to the found substring or to the end of `haystack`. + size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1]) + - &haystack_data[prev_haystack_offset]; + + if (pos != haystack_size) + { + res[i] = 1 + + Impl::countChars( + reinterpret_cast(&haystack_data[prev_haystack_offset]), + reinterpret_cast(&haystack_data[prev_haystack_offset + pos])); + } + else + res[i] = 0; + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Find many substrings in single string. + static void constantVector( + const String & haystack, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + PaddedPODArray & res) + { + // NOTE You could use haystack indexing. But this is a rare case. + + ColumnString::Offset prev_needle_offset = 0; + + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack( + reinterpret_cast(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1); + + size_t pos = searcher.search( + reinterpret_cast(haystack.data()), + reinterpret_cast(haystack.data()) + haystack.size()) + - reinterpret_cast(haystack.data()); + + if (pos != haystack.size()) + { + res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos); + } + else + res[i] = 0; + } + + prev_needle_offset = needle_offsets[i]; + } + } + + template + static void vectorFixedConstant(Args &&...) + { + throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/URL/cutURLParameter.cpp b/dbms/src/Functions/URL/cutURLParameter.cpp index b8f5c84fe83..f0103f42acd 100644 --- a/dbms/src/Functions/URL/cutURLParameter.cpp +++ b/dbms/src/Functions/URL/cutURLParameter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/dbms/src/Functions/URL/extractURLParameter.cpp b/dbms/src/Functions/URL/extractURLParameter.cpp index 8c5292bdc0a..c6234c66fc0 100644 --- a/dbms/src/Functions/URL/extractURLParameter.cpp +++ b/dbms/src/Functions/URL/extractURLParameter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/dbms/src/Functions/hasToken.cpp b/dbms/src/Functions/hasToken.cpp new file mode 100644 index 00000000000..ee04484ad54 --- /dev/null +++ b/dbms/src/Functions/hasToken.cpp @@ -0,0 +1,22 @@ +#include "FunctionsStringSearch.h" +#include +#include "HasTokenImpl.h" +#include + + +namespace DB +{ + +struct NameHasToken +{ + static constexpr auto name = "hasToken"; +}; + +using FunctionHasToken = FunctionsStringSearch, NameHasToken>; + +void registerFunctionHasToken(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/hasTokenCaseInsensitive.cpp b/dbms/src/Functions/hasTokenCaseInsensitive.cpp new file mode 100644 index 00000000000..da857f8b087 --- /dev/null +++ b/dbms/src/Functions/hasTokenCaseInsensitive.cpp @@ -0,0 +1,25 @@ +#include "FunctionsStringSearch.h" + +#include "FunctionsStringSearch.h" +#include +#include "HasTokenImpl.h" +#include + + +namespace DB +{ + +struct NameHasTokenCaseInsensitive +{ + static constexpr auto name = "hasTokenCaseInsensitive"; +}; + +using FunctionHasTokenCaseInsensitive + = FunctionsStringSearch, NameHasTokenCaseInsensitive>; + +void registerFunctionHasTokenCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositions.cpp b/dbms/src/Functions/multiSearchAllPositions.cpp new file mode 100644 index 00000000000..c7aeb4d6245 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositions.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositions +{ + static constexpr auto name = "multiSearchAllPositions"; +}; + +using FunctionMultiSearchAllPositions + = FunctionsMultiStringPosition, NameMultiSearchAllPositions>; + +void registerFunctionMultiSearchAllPositions(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp new file mode 100644 index 00000000000..4abcf7c8405 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsCaseInsensitive +{ + static constexpr auto name = "multiSearchAllPositionsCaseInsensitive"; +}; + +using FunctionMultiSearchAllPositionsCaseInsensitive + = FunctionsMultiStringPosition, NameMultiSearchAllPositionsCaseInsensitive>; + +void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..d9dbc1a7c8c --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsCaseInsensitiveUTF8.cpp @@ -0,0 +1,24 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition< + MultiSearchAllPositionsImpl, + NameMultiSearchAllPositionsCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp b/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp new file mode 100644 index 00000000000..8f39c0eade9 --- /dev/null +++ b/dbms/src/Functions/multiSearchAllPositionsUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringPosition.h" +#include "FunctionFactory.h" +#include "MultiSearchAllPositionsImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAllPositionsUTF8 +{ + static constexpr auto name = "multiSearchAllPositionsUTF8"; +}; + +using FunctionMultiSearchAllPositionsUTF8 + = FunctionsMultiStringPosition, NameMultiSearchAllPositionsUTF8>; + +void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAny.cpp b/dbms/src/Functions/multiSearchAny.cpp new file mode 100644 index 00000000000..144dbdbfdc4 --- /dev/null +++ b/dbms/src/Functions/multiSearchAny.cpp @@ -0,0 +1,22 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAny +{ + static constexpr auto name = "multiSearchAny"; +}; + +using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; + +void registerFunctionMultiSearchAny(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp b/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp new file mode 100644 index 00000000000..8b33a61013b --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp @@ -0,0 +1,22 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyCaseInsensitive +{ + static constexpr auto name = "multiSearchAnyCaseInsensitive"; +}; +using FunctionMultiSearchCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; + +void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..49a8b95a0e0 --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchCaseInsensitiveUTF8 + = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchAnyUTF8.cpp b/dbms/src/Functions/multiSearchAnyUTF8.cpp new file mode 100644 index 00000000000..55f2e449833 --- /dev/null +++ b/dbms/src/Functions/multiSearchAnyUTF8.cpp @@ -0,0 +1,21 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchAnyUTF8 +{ + static constexpr auto name = "multiSearchAnyUTF8"; +}; +using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; + +void registerFunctionMultiSearchAnyUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndex.cpp b/dbms/src/Functions/multiSearchFirstIndex.cpp new file mode 100644 index 00000000000..65a25004964 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndex.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndex +{ + static constexpr auto name = "multiSearchFirstIndex"; +}; + +using FunctionMultiSearchFirstIndex + = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; + +void registerFunctionMultiSearchFirstIndex(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp new file mode 100644 index 00000000000..bf643f0cf29 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexCaseInsensitive +{ + static constexpr auto name = "multiSearchFirstIndexCaseInsensitive"; +}; + +using FunctionMultiSearchFirstIndexCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; + +void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..005152388ab --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp b/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp new file mode 100644 index 00000000000..1158fda2a63 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstIndexUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstIndexImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstIndexUTF8 +{ + static constexpr auto name = "multiSearchFirstIndexUTF8"; +}; + +using FunctionMultiSearchFirstIndexUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; + +void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPosition.cpp b/dbms/src/Functions/multiSearchFirstPosition.cpp new file mode 100644 index 00000000000..06ac396250e --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPosition.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPosition +{ + static constexpr auto name = "multiSearchFirstPosition"; +}; + +using FunctionMultiSearchFirstPosition + = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; + +void registerFunctionMultiSearchFirstPosition(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp new file mode 100644 index 00000000000..1d028ad4513 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionCaseInsensitive +{ + static constexpr auto name = "multiSearchFirstPositionCaseInsensitive"; +}; + +using FunctionMultiSearchFirstPositionCaseInsensitive + = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; + +void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..0b355ddc446 --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp @@ -0,0 +1,24 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 +{ + static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8"; +}; + +using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch< + MultiSearchFirstPositionImpl, + NameMultiSearchFirstPositionCaseInsensitiveUTF8>; + +void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp b/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp new file mode 100644 index 00000000000..26e16d17e8d --- /dev/null +++ b/dbms/src/Functions/multiSearchFirstPositionUTF8.cpp @@ -0,0 +1,23 @@ +#include "FunctionsMultiStringSearch.h" +#include "FunctionFactory.h" +#include "MultiSearchFirstPositionImpl.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NameMultiSearchFirstPositionUTF8 +{ + static constexpr auto name = "multiSearchFirstPositionUTF8"; +}; + +using FunctionMultiSearchFirstPositionUTF8 + = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; + +void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/position.cpp b/dbms/src/Functions/position.cpp new file mode 100644 index 00000000000..df1405ae94a --- /dev/null +++ b/dbms/src/Functions/position.cpp @@ -0,0 +1,21 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePosition +{ + static constexpr auto name = "position"; +}; + +using FunctionPosition = FunctionsStringSearch, NamePosition>; + +void registerFunctionPosition(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive); +} +} diff --git a/dbms/src/Functions/positionCaseInsensitive.cpp b/dbms/src/Functions/positionCaseInsensitive.cpp new file mode 100644 index 00000000000..00721dda212 --- /dev/null +++ b/dbms/src/Functions/positionCaseInsensitive.cpp @@ -0,0 +1,20 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionCaseInsensitive +{ + static constexpr auto name = "positionCaseInsensitive"; +}; + +using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; + +void registerFunctionPositionCaseInsensitive(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp b/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..196a5d67cd2 --- /dev/null +++ b/dbms/src/Functions/positionCaseInsensitiveUTF8.cpp @@ -0,0 +1,22 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionCaseInsensitiveUTF8 +{ + static constexpr auto name = "positionCaseInsensitiveUTF8"; +}; + +using FunctionPositionCaseInsensitiveUTF8 + = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; + +void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/positionUTF8.cpp b/dbms/src/Functions/positionUTF8.cpp new file mode 100644 index 00000000000..944b55005f3 --- /dev/null +++ b/dbms/src/Functions/positionUTF8.cpp @@ -0,0 +1,21 @@ +#include "FunctionsStringSearch.h" +#include "FunctionFactory.h" +#include "PositionImpl.h" + + +namespace DB +{ + +struct NamePositionUTF8 +{ + static constexpr auto name = "positionUTF8"; +}; + +using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; + +void registerFunctionPositionUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsStringSearch.cpp b/dbms/src/Functions/registerFunctionsStringSearch.cpp new file mode 100644 index 00000000000..e4f89201491 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsStringSearch.cpp @@ -0,0 +1,66 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionPosition(FunctionFactory &); +void registerFunctionPositionUTF8(FunctionFactory &); +void registerFunctionPositionCaseInsensitive(FunctionFactory &); +void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchAny(FunctionFactory &); +void registerFunctionMultiSearchAnyUTF8(FunctionFactory &); +void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchFirstIndex(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchFirstPosition(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionMultiSearchAllPositions(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory &); +void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory &); + +void registerFunctionHasToken(FunctionFactory &); +void registerFunctionHasTokenCaseInsensitive(FunctionFactory &); + + +void registerFunctionsStringSearch(FunctionFactory & factory) +{ + registerFunctionPosition(factory); + registerFunctionPositionUTF8(factory); + registerFunctionPositionCaseInsensitive(factory); + registerFunctionPositionCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchAny(factory); + registerFunctionMultiSearchAnyUTF8(factory); + registerFunctionMultiSearchAnyCaseInsensitive(factory); + registerFunctionMultiSearchAnyCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchFirstIndex(factory); + registerFunctionMultiSearchFirstIndexUTF8(factory); + registerFunctionMultiSearchFirstIndexCaseInsensitive(factory); + registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchFirstPosition(factory); + registerFunctionMultiSearchFirstPositionUTF8(factory); + registerFunctionMultiSearchFirstPositionCaseInsensitive(factory); + registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(factory); + + registerFunctionMultiSearchAllPositions(factory); + registerFunctionMultiSearchAllPositionsUTF8(factory); + registerFunctionMultiSearchAllPositionsCaseInsensitive(factory); + registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(factory); + + registerFunctionHasToken(factory); + registerFunctionHasTokenCaseInsensitive(factory); +} + +} diff --git a/dbms/src/Functions/visitParamExtractRaw.cpp b/dbms/src/Functions/visitParamExtractRaw.cpp index 5eeb36286a5..e6e89f1c7ba 100644 --- a/dbms/src/Functions/visitParamExtractRaw.cpp +++ b/dbms/src/Functions/visitParamExtractRaw.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Functions/visitParamExtractString.cpp b/dbms/src/Functions/visitParamExtractString.cpp index a6f4b98145d..b633a59807e 100644 --- a/dbms/src/Functions/visitParamExtractString.cpp +++ b/dbms/src/Functions/visitParamExtractString.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB From 83f1fcaf37f97f89dc705f0ef036d8fe79b78dae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 10:50:47 +0300 Subject: [PATCH 117/147] Implement totalRows() for StorageBuffer --- dbms/src/Storages/StorageBuffer.cpp | 18 ++++++++++++++++++ dbms/src/Storages/StorageBuffer.h | 8 +++++--- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index bbbe194d768..36695e9f95e 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -725,6 +725,24 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S } } +std::optional StorageBuffer::totalRows() const +{ + std::optional underlying_rows; + auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id); + + if (underlying) + underlying_rows = underlying->totalRows(); + if (!underlying_rows) + return underlying_rows; + + UInt64 rows = 0; + for (auto & buffer : buffers) + { + std::lock_guard lock(buffer.mutex); + rows += buffer.data.rows(); + } + return rows + *underlying_rows; +} void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index df9fcbf750e..951eaf438b7 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -88,8 +88,10 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override; - /// The structure of the subordinate table is not checked and does not change. - void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + /// The structure of the subordinate table is not checked and does not change. + void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + + std::optional totalRows() const override; ~StorageBuffer() override; @@ -100,7 +102,7 @@ private: { time_t first_write_time = 0; Block data; - std::mutex mutex; + mutable std::mutex mutex; }; /// There are `num_shards` of independent buffers. From 611f565f211e5db9a662c0d540d2b0afc60e14cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:02:35 +0300 Subject: [PATCH 118/147] Implement totalRows() for StorageMemory --- dbms/src/Storages/StorageMemory.cpp | 9 +++++++++ dbms/src/Storages/StorageMemory.h | 4 +++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 1810a9b56e0..97e879b4606 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -136,6 +136,15 @@ void StorageMemory::truncate(const ASTPtr &, const Context &, TableStructureWrit data.clear(); } +std::optional StorageMemory::totalRows() const +{ + UInt64 rows = 0; + std::lock_guard lock(mutex); + for (auto & buffer : data) + rows += buffer.rows(); + return rows; +} + void registerStorageMemory(StorageFactory & factory) { diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 2515f948b4c..1541b95cc9d 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -42,11 +42,13 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + std::optional totalRows() const override; + private: /// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated. BlocksList data; - std::mutex mutex; + mutable std::mutex mutex; protected: StorageMemory(const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_); From b66f2efc9ed51929c76c2d7a5e449908695bd808 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 15:22:17 +0300 Subject: [PATCH 119/147] Implement totalRows() for StorageNull --- dbms/src/Storages/StorageNull.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index f34b9479a03..566fcd6fb4a 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -44,6 +44,11 @@ public: void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; + std::optional totalRows() const override + { + return {0}; + } + private: protected: From 2489481a46ac58c0dc87c591d81a9c3db02fec7c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:06:07 +0300 Subject: [PATCH 120/147] Add total_rows to the system.tables --- dbms/src/Storages/System/StorageSystemTables.cpp | 16 ++++++++++++++++ docs/en/operations/system_tables.md | 6 ++++++ 2 files changed, 22 insertions(+) diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index c4e6751f88b..603336ed358 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -49,6 +50,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) {"primary_key", std::make_shared()}, {"sampling_key", std::make_shared()}, {"storage_policy", std::make_shared()}, + {"total_rows", std::make_shared(std::make_shared())}, })); } @@ -204,6 +206,10 @@ protected: // storage_policy if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + + // total_rows + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); } } @@ -379,6 +385,16 @@ protected: else res_columns[res_index++]->insertDefault(); } + + if (columns_mask[src_index++]) + { + assert(table != nullptr); + auto total_rows = table->totalRows(); + if (total_rows) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } } } diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 4855dad0150..1f1eaa35cac 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -934,6 +934,12 @@ This table contains the following columns (the column type is shown in brackets) - `sorting_key` (String) - The sorting key expression specified in the table. - `primary_key` (String) - The primary key expression specified in the table. - `sampling_key` (String) - The sampling key expression specified in the table. +- `storage_policy` (String) - The storage policy: + + - [MergeTree](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes) + - [Distributed](table_engines/distributed.md#distributed) + +- `total_rows` (Nullable(UInt64)) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `Null` (including underying `Buffer` table). The `system.tables` table is used in `SHOW TABLES` query implementation. From c2fbba7b9c088e645976a05f4a07d950e123f61d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:22:08 +0300 Subject: [PATCH 121/147] Update 00753_system_columns_and_system_tables for storage_policy/total_rows --- ...system_columns_and_system_tables.reference | 16 +++++++-- ...00753_system_columns_and_system_tables.sql | 35 ++++++++++++++++++- 2 files changed, 47 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 6868492f7b0..4a5ae03d7da 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -1,11 +1,12 @@ -┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┐ -│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ -└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┘ +┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┬─storage_policy─┬─total_rows─┐ +│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ default │ 0 │ +└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┴────────────────┴────────────┘ ┌─name──┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐ │ name1 │ 0 │ 1 │ 1 │ 1 │ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ +1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ @@ -23,3 +24,12 @@ │ UserId │ 0 │ 1 │ 1 │ 1 │ │ Counter │ 0 │ 1 │ 1 │ 0 │ └─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ +Check total_rows for TinyLog +\N +\N +Check total_rows for Memory +0 +1 +Check total_rows for Buffer +0 +50 diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index 666f9c6c3f6..cca68d9d2a8 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -11,7 +11,7 @@ CREATE TABLE check_system_tables PARTITION BY name2 SAMPLE BY name1; -SELECT name, partition_key, sorting_key, primary_key, sampling_key +SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows FROM system.tables WHERE name = 'check_system_tables' FORMAT PrettyCompactNoEscapes; @@ -21,6 +21,9 @@ FROM system.columns WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; +INSERT INTO check_system_tables VALUES (1, 1, 1); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; + DROP TABLE IF EXISTS check_system_tables; -- Check VersionedCollapsingMergeTree @@ -65,3 +68,33 @@ WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; + +SELECT 'Check total_rows for TinyLog'; +CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog(); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +INSERT INTO check_system_tables VALUES (1); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +DROP TABLE check_system_tables; + +SELECT 'Check total_rows for Memory'; +CREATE TABLE check_system_tables (key UInt8) ENGINE = Memory(); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +INSERT INTO check_system_tables VALUES (1); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +DROP TABLE check_system_tables; + +SELECT 'Check total_rows for Buffer'; +CREATE TABLE check_system_tables_null (key UInt8) ENGINE = Null(); +CREATE TABLE check_system_tables (key UInt8) ENGINE = Buffer( + currentDatabase(), + check_system_tables_null, + 2, + 0, 100, /* min_time /max_time */ + 100, 100, /* min_rows /max_rows */ + 0, 1e6 /* min_bytes/max_bytes */ +); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); +SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +DROP TABLE check_system_tables; +DROP TABLE check_system_tables_null; From 4a7ee947b239958a5e9fce89bc5c7442c0a34638 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:38:38 +0300 Subject: [PATCH 122/147] Add IStorage::totalBytes() interface --- dbms/src/Storages/IStorage.h | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 497d7bbe08d..469f39d65df 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -459,13 +459,31 @@ public: /// Returns storage policy if storage supports it virtual StoragePolicyPtr getStoragePolicy() const { return {}; } - /** If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. - */ + /// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. + /// Used for: + /// - Simple count() opimization + /// - For total_rows column in system.tables + /// + /// Does takes underlying Storage (if any) into account. virtual std::optional totalRows() const { return {}; } + /// If it is possible to quickly determine exact number of bytes for the table on storage: + /// - memory (approximated) + /// - disk (compressed) + /// + /// Used for: + /// - For total_bytes column in system.tables + // + /// Does not takes underlying Storage (if any) into account + /// (since for Buffer we still need to know how much bytes it uses). + virtual std::optional totalBytes() const + { + return {}; + } + private: /// You always need to take the next three locks in this order. From 08d5872d9009f755017a4f2d4a723f4a8ca3e087 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:50:27 +0300 Subject: [PATCH 123/147] Implement totalBytes() for MergeTree/ReplicatedMergeTree --- dbms/src/Storages/StorageMergeTree.cpp | 5 +++++ dbms/src/Storages/StorageMergeTree.h | 1 + .../Storages/StorageReplicatedMergeTree.cpp | 19 ++++++++++++++++--- .../src/Storages/StorageReplicatedMergeTree.h | 4 ++++ 4 files changed, 26 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 79079aa3095..64950a47437 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -151,6 +151,11 @@ std::optional StorageMergeTree::totalRows() const return getTotalActiveSizeInRows(); } +std::optional StorageMergeTree::totalBytes() const +{ + return getTotalActiveSizeInBytes(); +} + BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Context & context) { return std::make_shared(*this, context.getSettingsRef().max_partitions_per_insert_block); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 93d7ac89832..8161e0b5c4b 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -46,6 +46,7 @@ public: unsigned num_streams) override; std::optional totalRows() const override; + std::optional totalBytes() const override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3d4a3bd9451..90c0d3418e6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2995,9 +2995,9 @@ Pipes StorageReplicatedMergeTree::read( } -std::optional StorageReplicatedMergeTree::totalRows() const +template +void StorageReplicatedMergeTree::foreachCommittedParts(const Func & func) const { - size_t res = 0; auto max_added_blocks = getMaxAddedBlocks(); auto lock = lockParts(); for (auto & part : getDataPartsStateRange(DataPartState::Committed)) @@ -3009,8 +3009,21 @@ std::optional StorageReplicatedMergeTree::totalRows() const if (blocks_iterator == max_added_blocks.end() || part->info.max_block > blocks_iterator->second) continue; - res += part->rows_count; + func(part); } +} + +std::optional StorageReplicatedMergeTree::totalRows() const +{ + UInt64 res = 0; + foreachCommittedParts([&res](auto & part) { res += part->rows_count; }); + return res; +} + +std::optional StorageReplicatedMergeTree::totalBytes() const +{ + UInt64 res = 0; + foreachCommittedParts([&res](auto & part) { res += part->getBytesOnDisk(); }); return res; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 1b92310b39d..b8132329ae4 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -96,6 +96,7 @@ public: unsigned num_streams) override; std::optional totalRows() const override; + std::optional totalBytes() const override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; @@ -287,6 +288,9 @@ private: /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; + template + void foreachCommittedParts(const Func & func) const; + /** Creates the minimum set of nodes in ZooKeeper. */ void createTableIfNotExists(); From bc32865e2251e16a71b02df0ffba0406acf43aff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:52:10 +0300 Subject: [PATCH 124/147] Implement totalBytes() for StorageMemory --- dbms/src/Storages/StorageMemory.cpp | 9 +++++++++ dbms/src/Storages/StorageMemory.h | 1 + 2 files changed, 10 insertions(+) diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 97e879b4606..f7af76e72b0 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -145,6 +145,15 @@ std::optional StorageMemory::totalRows() const return rows; } +std::optional StorageMemory::totalBytes() const +{ + UInt64 bytes = 0; + std::lock_guard lock(mutex); + for (auto & buffer : data) + bytes += buffer.bytes(); + return bytes; +} + void registerStorageMemory(StorageFactory & factory) { diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 1541b95cc9d..3a807540411 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -43,6 +43,7 @@ public: void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; std::optional totalRows() const override; + std::optional totalBytes() const override; private: /// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated. From ee89dfc92b1893c70c33a685749fed886f5721e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:54:00 +0300 Subject: [PATCH 125/147] Implement totalBytes() for StorageBuffer --- dbms/src/Storages/StorageBuffer.cpp | 11 +++++++++++ dbms/src/Storages/StorageBuffer.h | 1 + 2 files changed, 12 insertions(+) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 36695e9f95e..53fb257d58d 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -744,6 +744,17 @@ std::optional StorageBuffer::totalRows() const return rows + *underlying_rows; } +std::optional StorageBuffer::totalBytes() const +{ + UInt64 bytes = 0; + for (auto & buffer : buffers) + { + std::lock_guard lock(buffer.mutex); + bytes += buffer.data.bytes(); + } + return bytes; +} + void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) { lockStructureExclusively(table_lock_holder, context.getCurrentQueryId()); diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 951eaf438b7..7a3d907ae76 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -92,6 +92,7 @@ public: void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override; std::optional totalRows() const override; + std::optional totalBytes() const override; ~StorageBuffer() override; From fe9a6385404399fa36e362b889ee9dd24d00fdd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 17:44:32 +0300 Subject: [PATCH 126/147] Implement totalBytes() for StorageNull --- dbms/src/Storages/StorageNull.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index 566fcd6fb4a..d9aeb60c260 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -48,6 +48,10 @@ public: { return {0}; } + std::optional totalBytes() const override + { + return {0}; + } private: From 997c4682aa8114d15cbe7921f970dd7db859cdec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 11:58:39 +0300 Subject: [PATCH 127/147] Add total_bytes to the system.tables --- dbms/src/Storages/System/StorageSystemTables.cpp | 15 +++++++++++++++ docs/en/operations/system_tables.md | 4 ++++ 2 files changed, 19 insertions(+) diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 603336ed358..5d0aec921de 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -51,6 +51,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) {"sampling_key", std::make_shared()}, {"storage_policy", std::make_shared()}, {"total_rows", std::make_shared(std::make_shared())}, + {"total_bytes", std::make_shared(std::make_shared())}, })); } @@ -210,6 +211,10 @@ protected: // total_rows if (columns_mask[src_index++]) res_columns[res_index++]->insertDefault(); + + // total_bytes + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); } } @@ -395,6 +400,16 @@ protected: else res_columns[res_index++]->insertDefault(); } + + if (columns_mask[src_index++]) + { + assert(table != nullptr); + auto total_bytes = table->totalBytes(); + if (total_bytes) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } } } diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 1f1eaa35cac..1ef90001ae1 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -940,6 +940,10 @@ This table contains the following columns (the column type is shown in brackets) - [Distributed](table_engines/distributed.md#distributed) - `total_rows` (Nullable(UInt64)) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `Null` (including underying `Buffer` table). +- `total_bytes` (Nullable(UInt64)) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `Null` (**does not** includes any underlying storage). + + - If the table stores data on disk, returns used space on disk (i.e. compressed). + - If the table stores data in memory, returns approximated number of used bytes in memory. The `system.tables` table is used in `SHOW TABLES` query implementation. From 40f0acfd526e6d60bd55a97ebfab0eb8714c4548 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 12:00:24 +0300 Subject: [PATCH 128/147] Update 00753_system_columns_and_system_tables for total_bytes --- ...system_columns_and_system_tables.reference | 20 +++++++------- ...00753_system_columns_and_system_tables.sql | 26 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 4a5ae03d7da..e84e610bb93 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -1 +231 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ │ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ @@ -24,12 +24,12 @@ │ UserId │ 0 │ 1 │ 1 │ 1 │ │ Counter │ 0 │ 1 │ 1 │ 0 │ └─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -Check total_rows for TinyLog -\N -\N -Check total_rows for Memory -0 -1 -Check total_rows for Buffer -0 -50 +Check total_bytes/total_rows for TinyLog +\N \N +\N \N +Check total_bytes/total_rows for Memory +0 0 +2 1 +Check total_bytes/total_rows for Buffer +0 0 +100 50 diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index cca68d9d2a8..7c17657afb6 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -22,7 +22,7 @@ WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; INSERT INTO check_system_tables VALUES (1, 1, 1); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; DROP TABLE IF EXISTS check_system_tables; @@ -69,23 +69,23 @@ FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS check_system_tables; -SELECT 'Check total_rows for TinyLog'; +SELECT 'Check total_bytes/total_rows for TinyLog'; CREATE TABLE check_system_tables (key UInt8) ENGINE = TinyLog(); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; INSERT INTO check_system_tables VALUES (1); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; DROP TABLE check_system_tables; -SELECT 'Check total_rows for Memory'; -CREATE TABLE check_system_tables (key UInt8) ENGINE = Memory(); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT 'Check total_bytes/total_rows for Memory'; +CREATE TABLE check_system_tables (key UInt16) ENGINE = Memory(); +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; INSERT INTO check_system_tables VALUES (1); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; DROP TABLE check_system_tables; -SELECT 'Check total_rows for Buffer'; -CREATE TABLE check_system_tables_null (key UInt8) ENGINE = Null(); -CREATE TABLE check_system_tables (key UInt8) ENGINE = Buffer( +SELECT 'Check total_bytes/total_rows for Buffer'; +CREATE TABLE check_system_tables_null (key UInt16) ENGINE = Null(); +CREATE TABLE check_system_tables (key UInt16) ENGINE = Buffer( currentDatabase(), check_system_tables_null, 2, @@ -93,8 +93,8 @@ CREATE TABLE check_system_tables (key UInt8) ENGINE = Buffer( 100, 100, /* min_rows /max_rows */ 0, 1e6 /* min_bytes/max_bytes */ ); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; INSERT INTO check_system_tables SELECT * FROM numbers_mt(50); -SELECT total_rows FROM system.tables WHERE name = 'check_system_tables'; +SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables'; DROP TABLE check_system_tables; DROP TABLE check_system_tables_null; From 85df027651b7c4c578d18994b95424c6f671412c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 17:58:21 +0300 Subject: [PATCH 129/147] Update 00693_max_block_size_system_tables_columns for total_rows/total_bytes --- .../00693_max_block_size_system_tables_columns.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.reference b/dbms/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.reference index 702b59b8258..b674529e010 100644 --- a/dbms/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.reference +++ b/dbms/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.reference @@ -2,7 +2,7 @@ 1 1 1 - t_00693 Memory 1 [] 0000-00-00 00:00:00 [] [] Memory + t_00693 Memory 1 [] 0000-00-00 00:00:00 [] [] Memory \N \N 1 1 1 From 3d3a82543a6ccaa7a74e49007751d5d115d61f08 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Mar 2020 18:11:03 +0300 Subject: [PATCH 130/147] Fix optimize_trivial_count_query for queries with GROUP BY Fixes: 00341_squashing_insert_select2 --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 3e4fb7d5909..6a6ee102e70 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1064,7 +1064,7 @@ void InterpreterSelectQuery::executeFetchColumns( auto check_trivial_count_query = [&]() -> std::optional { if (!settings.optimize_trivial_count_query || !syntax_analyzer_result->maybe_optimize_trivial_count || !storage - || query.sampleSize() || query.sampleOffset() || query.final() || query.prewhere() || query.where() + || query.sampleSize() || query.sampleOffset() || query.final() || query.prewhere() || query.where() || query.groupBy() || !query_analyzer->hasAggregation() || processing_stage != QueryProcessingStage::FetchColumns) return {}; From d2edeb7be4965ea5de7db322155b8754b1595d38 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 30 Mar 2020 02:22:06 +0300 Subject: [PATCH 131/147] Fix parsing multiple hosts in CREATE USER command. --- dbms/src/Parsers/ParserCreateUserQuery.cpp | 21 ++++++++++++------- .../01075_allowed_client_hosts.reference | 3 +++ .../01075_allowed_client_hosts.sql | 9 ++++++++ 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/dbms/src/Parsers/ParserCreateUserQuery.cpp b/dbms/src/Parsers/ParserCreateUserQuery.cpp index a7cc6550644..57f50c34116 100644 --- a/dbms/src/Parsers/ParserCreateUserQuery.cpp +++ b/dbms/src/Parsers/ParserCreateUserQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -168,34 +169,38 @@ namespace else if (ParserKeyword{"NAME REGEXP"}.ignore(pos, expected)) { ASTPtr ast; - if (!ParserStringLiteral{}.parse(pos, ast, expected)) + if (!ParserList{std::make_unique(), std::make_unique(TokenType::Comma), false}.parse(pos, ast, expected)) return false; - new_hosts.addNameRegexp(ast->as().value.safeGet()); + for (const auto & name_regexp_ast : ast->children) + new_hosts.addNameRegexp(name_regexp_ast->as().value.safeGet()); } else if (ParserKeyword{"NAME"}.ignore(pos, expected)) { ASTPtr ast; - if (!ParserStringLiteral{}.parse(pos, ast, expected)) + if (!ParserList{std::make_unique(), std::make_unique(TokenType::Comma), false}.parse(pos, ast, expected)) return false; - new_hosts.addName(ast->as().value.safeGet()); + for (const auto & name_ast : ast->children) + new_hosts.addName(name_ast->as().value.safeGet()); } else if (ParserKeyword{"IP"}.ignore(pos, expected)) { ASTPtr ast; - if (!ParserStringLiteral{}.parse(pos, ast, expected)) + if (!ParserList{std::make_unique(), std::make_unique(TokenType::Comma), false}.parse(pos, ast, expected)) return false; - new_hosts.addSubnet(ast->as().value.safeGet()); + for (const auto & subnet_ast : ast->children) + new_hosts.addSubnet(subnet_ast->as().value.safeGet()); } else if (ParserKeyword{"LIKE"}.ignore(pos, expected)) { ASTPtr ast; - if (!ParserStringLiteral{}.parse(pos, ast, expected)) + if (!ParserList{std::make_unique(), std::make_unique(TokenType::Comma), false}.parse(pos, ast, expected)) return false; - new_hosts.addLikePattern(ast->as().value.safeGet()); + for (const auto & pattern_ast : ast->children) + new_hosts.addLikePattern(pattern_ast->as().value.safeGet()); } else return false; diff --git a/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.reference b/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.reference index dadef736cce..0082653059c 100644 --- a/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.reference +++ b/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.reference @@ -9,6 +9,9 @@ CREATE USER test_user_01075 HOST LOCAL CREATE USER test_user_01075 HOST NONE CREATE USER test_user_01075 HOST LIKE \'@.somesite.com\' CREATE USER test_user_01075 HOST NAME REGEXP \'.*.anothersite.com\' +CREATE USER test_user_01075 HOST NAME REGEXP \'.*.anothersite.com\', \'.*.anothersite.org\' +CREATE USER test_user_01075 HOST NAME REGEXP \'.*.anothersite2.com\', \'.*.anothersite2.org\' +CREATE USER test_user_01075 HOST NAME REGEXP \'.*.anothersite3.com\', \'.*.anothersite3.org\' CREATE USER `test_user_01075_x@localhost` HOST LOCAL CREATE USER test_user_01075_x CREATE USER `test_user_01075_x@192.168.23.15` HOST LIKE \'192.168.23.15\' diff --git a/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.sql b/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.sql index 6ffd8caf746..77a16a9f62a 100644 --- a/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.sql +++ b/dbms/tests/queries/0_stateless/01075_allowed_client_hosts.sql @@ -33,6 +33,15 @@ SHOW CREATE USER test_user_01075; ALTER USER test_user_01075 HOST NAME REGEXP '.*\.anothersite\.com'; SHOW CREATE USER test_user_01075; +ALTER USER test_user_01075 HOST NAME REGEXP '.*\.anothersite\.com', '.*\.anothersite\.org'; +SHOW CREATE USER test_user_01075; + +ALTER USER test_user_01075 HOST NAME REGEXP '.*\.anothersite2\.com', NAME REGEXP '.*\.anothersite2\.org'; +SHOW CREATE USER test_user_01075; + +ALTER USER test_user_01075 HOST NAME REGEXP '.*\.anothersite3\.com' HOST NAME REGEXP '.*\.anothersite3\.org'; +SHOW CREATE USER test_user_01075; + DROP USER test_user_01075; CREATE USER test_user_01075_x@localhost; From cc60f0edaf0db8d175f50d8c29b1499d59e9c270 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 30 Mar 2020 11:25:29 +0300 Subject: [PATCH 132/147] Custom mkdocs theme (#9874) --- docs/es/changelog/2017.md | 4 + docs/es/changelog/2018.md | 1059 +- docs/es/changelog/2019.md | 2066 ++- docs/es/changelog/index.md | 628 +- docs/es/commercial/cloud.md | 4 + docs/es/data_types/array.md | 4 + docs/es/data_types/boolean.md | 4 + docs/es/data_types/date.md | 4 + docs/es/data_types/datetime.md | 4 + docs/es/data_types/datetime64.md | 4 + docs/es/data_types/decimal.md | 4 + docs/es/data_types/domains/ipv4.md | 4 + docs/es/data_types/domains/ipv6.md | 4 + docs/es/data_types/domains/overview.md | 4 + docs/es/data_types/enum.md | 4 + docs/es/data_types/fixedstring.md | 4 + docs/es/data_types/float.md | 4 + docs/es/data_types/index.md | 4 + docs/es/data_types/int_uint.md | 4 + .../aggregatefunction.md | 4 + .../nested_data_structures/index.md | 4 + .../nested_data_structures/nested.md | 4 + docs/es/data_types/nullable.md | 4 + .../special_data_types/expression.md | 4 + .../es/data_types/special_data_types/index.md | 4 + .../data_types/special_data_types/interval.md | 4 + .../data_types/special_data_types/nothing.md | 4 + docs/es/data_types/special_data_types/set.md | 4 + docs/es/data_types/string.md | 4 + docs/es/data_types/tuple.md | 4 + docs/es/data_types/uuid.md | 4 + docs/es/database_engines/index.md | 4 + docs/es/database_engines/lazy.md | 4 + docs/es/database_engines/mysql.md | 4 + docs/es/development/architecture.md | 4 + docs/es/development/browse_code.md | 4 + docs/es/development/build.md | 4 + docs/es/development/build_cross_arm.md | 4 + docs/es/development/build_cross_osx.md | 4 + docs/es/development/build_osx.md | 4 + docs/es/development/contrib.md | 4 + docs/es/development/developer_instruction.md | 4 + docs/es/development/index.md | 4 + docs/es/development/style.md | 4 + docs/es/development/tests.md | 4 + docs/es/faq/general.md | 4 + .../example_datasets/amplab_benchmark.md | 4 + .../example_datasets/criteo.md | 4 + .../getting_started/example_datasets/index.md | 1 + .../example_datasets/metrica.md | 4 + .../example_datasets/nyc_taxi.md | 4 + .../example_datasets/ontime.md | 4 + .../example_datasets/star_schema.md | 4 + .../example_datasets/wikistat.md | 4 + docs/es/getting_started/index.md | 4 + docs/es/getting_started/install.md | 4 + docs/es/getting_started/playground.md | 1 + docs/es/getting_started/tutorial.md | 4 + docs/es/guides/apply_catboost_model.md | 4 + docs/es/guides/index.md | 4 + docs/es/index.md | 4 + docs/es/interfaces/cli.md | 4 + docs/es/interfaces/cpp.md | 4 + docs/es/interfaces/formats.md | 1210 +- docs/es/interfaces/http.md | 4 + docs/es/interfaces/index.md | 4 + docs/es/interfaces/jdbc.md | 4 + docs/es/interfaces/mysql.md | 4 + docs/es/interfaces/odbc.md | 4 + docs/es/interfaces/tcp.md | 4 + .../third-party/client_libraries.md | 4 + docs/es/interfaces/third-party/gui.md | 4 + .../es/interfaces/third-party/integrations.md | 4 + docs/es/interfaces/third-party/proxy.md | 4 + docs/es/introduction/adopters.md | 4 + docs/es/introduction/distinctive_features.md | 4 + .../features_considered_disadvantages.md | 4 + docs/es/introduction/history.md | 4 + docs/es/introduction/performance.md | 4 + docs/es/operations/access_rights.md | 4 + docs/es/operations/backup.md | 4 + docs/es/operations/configuration_files.md | 4 + docs/es/operations/index.md | 4 + docs/es/operations/monitoring.md | 4 + .../performance/sampling_query_profiler.md | 4 + docs/es/operations/performance_test.md | 4 + docs/es/operations/quotas.md | 4 + docs/es/operations/requirements.md | 4 + docs/es/operations/server_settings/index.md | 4 + .../es/operations/server_settings/settings.md | 4 + .../settings/constraints_on_settings.md | 4 + docs/es/operations/settings/index.md | 4 + .../settings/permissions_for_queries.md | 4 + .../operations/settings/query_complexity.md | 4 + docs/es/operations/settings/settings.md | 4 + .../operations/settings/settings_profiles.md | 4 + docs/es/operations/settings/settings_users.md | 4 + docs/es/operations/system_tables.md | 4 + .../table_engines/aggregatingmergetree.md | 4 + docs/es/operations/table_engines/buffer.md | 4 + .../table_engines/collapsingmergetree.md | 4 + .../table_engines/custom_partitioning_key.md | 4 + .../es/operations/table_engines/dictionary.md | 4 + .../operations/table_engines/distributed.md | 4 + .../operations/table_engines/external_data.md | 4 + docs/es/operations/table_engines/file.md | 4 + docs/es/operations/table_engines/generate.md | 4 + .../table_engines/graphitemergetree.md | 4 + docs/es/operations/table_engines/hdfs.md | 4 + docs/es/operations/table_engines/index.md | 4 + docs/es/operations/table_engines/jdbc.md | 4 + docs/es/operations/table_engines/join.md | 4 + docs/es/operations/table_engines/kafka.md | 4 + docs/es/operations/table_engines/log.md | 4 + .../es/operations/table_engines/log_family.md | 4 + .../table_engines/materializedview.md | 4 + docs/es/operations/table_engines/memory.md | 4 + docs/es/operations/table_engines/merge.md | 4 + docs/es/operations/table_engines/mergetree.md | 4 + docs/es/operations/table_engines/mysql.md | 4 + docs/es/operations/table_engines/null.md | 4 + docs/es/operations/table_engines/odbc.md | 4 + .../table_engines/replacingmergetree.md | 4 + .../operations/table_engines/replication.md | 4 + docs/es/operations/table_engines/set.md | 4 + docs/es/operations/table_engines/stripelog.md | 4 + .../table_engines/summingmergetree.md | 4 + docs/es/operations/table_engines/tinylog.md | 4 + docs/es/operations/table_engines/url.md | 4 + .../versionedcollapsingmergetree.md | 4 + docs/es/operations/table_engines/view.md | 4 + docs/es/operations/tips.md | 4 + docs/es/operations/troubleshooting.md | 4 + docs/es/operations/update.md | 4 + .../operations/utils/clickhouse-benchmark.md | 4 + docs/es/operations/utils/clickhouse-copier.md | 4 + docs/es/operations/utils/clickhouse-local.md | 4 + docs/es/operations/utils/index.md | 4 + .../agg_functions/combinators.md | 4 + docs/es/query_language/agg_functions/index.md | 4 + .../agg_functions/parametric_functions.md | 4 + .../query_language/agg_functions/reference.md | 1835 ++- docs/es/query_language/alter.md | 4 + docs/es/query_language/create.md | 4 + .../es/query_language/dicts/external_dicts.md | 4 + .../dicts/external_dicts_dict.md | 4 + .../dicts/external_dicts_dict_hierarchical.md | 4 + .../dicts/external_dicts_dict_layout.md | 4 + .../dicts/external_dicts_dict_lifetime.md | 4 + .../dicts/external_dicts_dict_sources.md | 4 + .../dicts/external_dicts_dict_structure.md | 4 + docs/es/query_language/dicts/index.md | 4 + .../es/query_language/dicts/internal_dicts.md | 4 + .../functions/arithmetic_functions.md | 4 + .../functions/array_functions.md | 4 + .../es/query_language/functions/array_join.md | 4 + .../query_language/functions/bit_functions.md | 4 + .../functions/bitmap_functions.md | 4 + .../functions/comparison_functions.md | 4 + .../functions/conditional_functions.md | 4 + .../functions/date_time_functions.md | 4 + .../functions/encoding_functions.md | 4 + .../functions/ext_dict_functions.md | 4 + .../functions/functions_for_nulls.md | 4 + docs/es/query_language/functions/geo.md | 4 + .../functions/hash_functions.md | 4 + .../functions/higher_order_functions.md | 4 + .../query_language/functions/in_functions.md | 4 + docs/es/query_language/functions/index.md | 4 + .../query_language/functions/introspection.md | 4 + .../functions/ip_address_functions.md | 4 + .../functions/json_functions.md | 4 + .../functions/logical_functions.md | 4 + .../functions/machine_learning_functions.md | 4 + .../functions/math_functions.md | 4 + .../functions/other_functions.md | 4 + .../functions/random_functions.md | 4 + .../functions/rounding_functions.md | 4 + .../functions/splitting_merging_functions.md | 4 + .../functions/string_functions.md | 4 + .../functions/string_replace_functions.md | 4 + .../functions/string_search_functions.md | 4 + .../functions/type_conversion_functions.md | 4 + .../query_language/functions/url_functions.md | 4 + .../functions/uuid_functions.md | 4 + .../functions/ym_dict_functions.md | 4 + docs/es/query_language/index.md | 4 + docs/es/query_language/insert_into.md | 4 + docs/es/query_language/misc.md | 4 + docs/es/query_language/operators.md | 4 + docs/es/query_language/select.md | 4 + docs/es/query_language/show.md | 4 + docs/es/query_language/syntax.md | 4 + docs/es/query_language/system.md | 4 + .../es/query_language/table_functions/file.md | 4 + .../table_functions/generate.md | 4 + .../es/query_language/table_functions/hdfs.md | 4 + .../query_language/table_functions/index.md | 4 + .../query_language/table_functions/input.md | 4 + .../es/query_language/table_functions/jdbc.md | 4 + .../query_language/table_functions/merge.md | 4 + .../query_language/table_functions/mysql.md | 4 + .../query_language/table_functions/numbers.md | 4 + .../es/query_language/table_functions/odbc.md | 4 + .../query_language/table_functions/remote.md | 4 + docs/es/query_language/table_functions/url.md | 4 + docs/es/roadmap.md | 4 + docs/es/security_changelog.md | 4 + docs/fa/getting_started/playground.md | 1 + docs/ja/getting_started/playground.md | 1 + docs/tools/build.py | 123 +- docs/tools/concatenate.py | 10 +- docs/tools/github.py | 13 +- docs/tools/make_links.sh | 2 +- docs/tools/mdx_clickhouse.py | 35 +- .../mkdocs-material-theme/partials/nav.html | 2 +- .../partials/social.html | 2 +- docs/tools/release.sh | 4 +- docs/tools/requirements.txt | 2 + docs/tools/test.py | 2 +- docs/tools/translate/babel-mapping.ini | 3 + docs/tools/translate/filter.py | 25 +- docs/tools/translate/requirements.txt | 2 + docs/tools/translate/translate.py | 36 +- docs/tools/translate/update-po.sh | 22 + docs/tools/website.py | 73 +- docs/zh/getting_started/playground.md | 1 + website/404.html | 6 +- website/css/base.css | 59 +- website/css/bootstrap.css | 10224 +++++++++++++++ website/css/docs.css | 186 + website/css/docsearch.css | 556 + website/images/flags/en.svg | 1 + website/images/flags/es.svg | 581 + website/images/flags/fa.svg | 1 + website/images/flags/ja.svg | 1 + website/images/flags/ru.svg | 1 + website/images/flags/zh.svg | 1 + website/images/index/github.svg | 2 +- website/images/mkdocs/edit.svg | 3 + website/images/mkdocs/multi.svg | 3 + website/images/mkdocs/pdf.svg | 3 + website/images/mkdocs/search.svg | 3 + website/images/mkdocs/single.svg | 3 + website/js/base.js | 42 +- website/js/bootstrap.js | 4521 +++++++ website/js/docs.js | 93 + website/js/docsearch.js | 10137 +++++++++++++++ website/js/externs.js | 6 + website/js/jquery-3.4.1.js | 10598 ++++++++++++++++ website/js/popper.js | 2624 ++++ website/locale/en/LC_MESSAGES/messages.mo | Bin 0 -> 443 bytes website/locale/en/LC_MESSAGES/messages.po | 266 + website/locale/es/LC_MESSAGES/messages.mo | Bin 0 -> 6235 bytes website/locale/es/LC_MESSAGES/messages.po | 274 + website/locale/fa/LC_MESSAGES/messages.mo | Bin 0 -> 6903 bytes website/locale/fa/LC_MESSAGES/messages.po | 266 + website/locale/ja/LC_MESSAGES/messages.mo | Bin 0 -> 6205 bytes website/locale/ja/LC_MESSAGES/messages.po | 253 + website/locale/messages.pot | 244 + website/locale/ru/LC_MESSAGES/messages.mo | Bin 0 -> 7844 bytes website/locale/ru/LC_MESSAGES/messages.po | 271 + website/locale/zh/LC_MESSAGES/messages.mo | Bin 0 -> 5610 bytes website/locale/zh/LC_MESSAGES/messages.po | 253 + website/main.html | 41 + website/templates/base.html | 7 +- website/templates/common_css.html | 5 +- website/templates/common_js.html | 9 +- website/templates/common_meta.html | 16 +- website/templates/docs/content.html | 34 + website/templates/docs/nav.html | 64 + website/templates/docs/sidebar-item.html | 18 + website/templates/docs/sidebar.html | 43 + website/templates/docs/toc-item.html | 13 + website/templates/docs/toc.html | 16 + website/templates/footer.html | 11 +- website/templates/index/community.html | 99 +- website/templates/index/efficient.html | 4 +- website/templates/index/events.html | 4 +- website/templates/index/hero.html | 6 +- website/templates/index/nav.html | 11 +- website/templates/index/performance.html | 18 +- website/templates/index/quickstart.html | 4 +- website/templates/index/reliable.html | 4 +- website/templates/index/success.html | 10 +- website/templates/index/why.html | 8 +- 286 files changed, 49659 insertions(+), 231 deletions(-) mode change 120000 => 100644 docs/es/changelog/2018.md mode change 120000 => 100644 docs/es/changelog/2019.md mode change 120000 => 100644 docs/es/changelog/index.md create mode 120000 docs/es/getting_started/example_datasets/index.md create mode 120000 docs/es/getting_started/playground.md mode change 120000 => 100644 docs/es/interfaces/formats.md mode change 120000 => 100644 docs/es/query_language/agg_functions/reference.md create mode 120000 docs/fa/getting_started/playground.md create mode 120000 docs/ja/getting_started/playground.md create mode 100644 docs/tools/translate/babel-mapping.ini create mode 100755 docs/tools/translate/update-po.sh create mode 120000 docs/zh/getting_started/playground.md create mode 100644 website/css/bootstrap.css create mode 100644 website/css/docs.css create mode 100644 website/css/docsearch.css create mode 100644 website/images/flags/en.svg create mode 100644 website/images/flags/es.svg create mode 100644 website/images/flags/fa.svg create mode 100644 website/images/flags/ja.svg create mode 100644 website/images/flags/ru.svg create mode 100644 website/images/flags/zh.svg create mode 100644 website/images/mkdocs/edit.svg create mode 100644 website/images/mkdocs/multi.svg create mode 100644 website/images/mkdocs/pdf.svg create mode 100644 website/images/mkdocs/search.svg create mode 100644 website/images/mkdocs/single.svg create mode 100644 website/js/bootstrap.js create mode 100644 website/js/docs.js create mode 100644 website/js/docsearch.js create mode 100644 website/js/externs.js create mode 100644 website/js/jquery-3.4.1.js create mode 100644 website/js/popper.js create mode 100644 website/locale/en/LC_MESSAGES/messages.mo create mode 100644 website/locale/en/LC_MESSAGES/messages.po create mode 100644 website/locale/es/LC_MESSAGES/messages.mo create mode 100644 website/locale/es/LC_MESSAGES/messages.po create mode 100644 website/locale/fa/LC_MESSAGES/messages.mo create mode 100644 website/locale/fa/LC_MESSAGES/messages.po create mode 100644 website/locale/ja/LC_MESSAGES/messages.mo create mode 100644 website/locale/ja/LC_MESSAGES/messages.po create mode 100644 website/locale/messages.pot create mode 100644 website/locale/ru/LC_MESSAGES/messages.mo create mode 100644 website/locale/ru/LC_MESSAGES/messages.po create mode 100644 website/locale/zh/LC_MESSAGES/messages.mo create mode 100644 website/locale/zh/LC_MESSAGES/messages.po create mode 100644 website/main.html create mode 100644 website/templates/docs/content.html create mode 100644 website/templates/docs/nav.html create mode 100644 website/templates/docs/sidebar-item.html create mode 100644 website/templates/docs/sidebar.html create mode 100644 website/templates/docs/toc-item.html create mode 100644 website/templates/docs/toc.html diff --git a/docs/es/changelog/2017.md b/docs/es/changelog/2017.md index 1f443076151..b80e820169d 100644 --- a/docs/es/changelog/2017.md +++ b/docs/es/changelog/2017.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ### Lanzamiento de ClickHouse 1.1.54327, 2017-12-21 {#clickhouse-release-1-1-54327-2017-12-21} Esta versión contiene correcciones de errores para la versión anterior 1.1.54318: diff --git a/docs/es/changelog/2018.md b/docs/es/changelog/2018.md deleted file mode 120000 index 20799251f43..00000000000 --- a/docs/es/changelog/2018.md +++ /dev/null @@ -1 +0,0 @@ -../../en/changelog/2018.md \ No newline at end of file diff --git a/docs/es/changelog/2018.md b/docs/es/changelog/2018.md new file mode 100644 index 00000000000..7aa67e7934e --- /dev/null +++ b/docs/es/changelog/2018.md @@ -0,0 +1,1058 @@ +--- +machine_translated: true +--- + +## Lanzamiento de ClickHouse 18.16 {#clickhouse-release-18-16} + +### Lanzamiento de ClickHouse 18.16.1, 2018-12-21 {#clickhouse-release-18-16-1-2018-12-21} + +#### Corrección de errores: {#bug-fixes} + +- Se corrigió un error que provocaba problemas con la actualización de diccionarios con el origen ODBC. [\#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [\#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) +- La compilación JIT de funciones agregadas ahora funciona con columnas LowCardinality. [\#3838](https://github.com/ClickHouse/ClickHouse/issues/3838) + +#### Mejora: {#improvements} + +- Se agregó el `low_cardinality_allow_in_native_format` configuración (habilitada por defecto). Cuando se deshabilita, las columnas LowCardinality se convertirán en columnas ordinarias para las consultas SELECT y se esperarán columnas ordinarias para las consultas INSERT. [\#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) + +#### Mejoras de construcción: {#build-improvements} + +- Correcciones para compilaciones en macOS y ARM. + +### Lanzamiento de ClickHouse 18.16.0, 2018-12-14 {#clickhouse-release-18-16-0-2018-12-14} + +#### Novedad: {#new-features} + +- `DEFAULT` Las expresiones se evalúan en busca de campos faltantes al cargar datos en formatos de entrada semiestructurados (`JSONEachRow`, `TSKV`). La función está habilitada con `insert_sample_with_metadata` configuración. [\#3555](https://github.com/ClickHouse/ClickHouse/pull/3555) +- El `ALTER TABLE` consulta ahora tiene el `MODIFY ORDER BY` acción para cambiar la clave de ordenación al agregar o eliminar una columna de tabla. Esto es útil para las tablas `MergeTree` familia que realizan tareas adicionales al fusionarse basándose en esta clave de ordenación, como `SummingMergeTree`, `AggregatingMergeTree` y así sucesivamente. [\#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) [\#3755](https://github.com/ClickHouse/ClickHouse/pull/3755) +- Para las tablas en el `MergeTree` familia, ahora puede especificar una clave de clasificación diferente (`ORDER BY`) e índice (`PRIMARY KEY`). La clave de ordenación puede ser más larga que el índice. [\#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) +- Se agregó el `hdfs` función de la tabla y el `HDFS` para importar y exportar datos a HDFS. [Más información](https://github.com/ClickHouse/ClickHouse/pull/3617) +- Funciones añadidas para trabajar con base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alejandro Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3350) +- Ahora puede usar un parámetro para configurar la precisión del `uniqCombined` función de agregado (seleccione el número de celdas HyperLogLog). [\#3406](https://github.com/ClickHouse/ClickHouse/pull/3406) +- Se agregó el `system.contributors` tabla que contiene los nombres de todos los que hicieron compromete en ClickHouse. [\#3452](https://github.com/ClickHouse/ClickHouse/pull/3452) +- Se agregó la capacidad de omitir la partición para el `ALTER TABLE ... FREEZE` consulta para hacer una copia de seguridad de todas las particiones a la vez. [\#3514](https://github.com/ClickHouse/ClickHouse/pull/3514) +- Añadir `dictGet` y `dictGetOrDefault` funciones que no requieren especificar el tipo de valor de retorno. El tipo se determina automáticamente a partir de la descripción del diccionario. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3564) +- Ahora puede especificar comentarios para una columna en la descripción de la tabla y cambiarla usando `ALTER`. [\#3377](https://github.com/ClickHouse/ClickHouse/pull/3377) +- La lectura es compatible para `Join` tablas de escribir con teclas simples. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3728) +- Ahora puede especificar las opciones `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, y `join_overflow_mode` al crear un `Join` tabla de tipo. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3728) +- Se agregó el `joinGet` función que le permite utilizar un `Join` tipo de tabla como un diccionario. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3728) +- Se agregó el `partition_key`, `sorting_key`, `primary_key`, y `sampling_key` columnas a la `system.tables` tabla con el fin de proporcionar información sobre las claves de la tabla. [\#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) +- Se agregó el `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, y `is_in_sampling_key` columnas a la `system.columns` tabla. [\#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) +- Se agregó el `min_time` y `max_time` columnas a la `system.parts` tabla. Estas columnas se rellenan cuando la clave de partición es una expresión que consiste en `DateTime` columna. [Emmanuel Donin de Rosière](https://github.com/ClickHouse/ClickHouse/pull/3800) + +#### Corrección de errores: {#bug-fixes-1} + +- Correcciones y mejoras de rendimiento para el `LowCardinality` tipo de datos. `GROUP BY` utilizar `LowCardinality(Nullable(...))`. Obtener los valores de `extremes`. Procesamiento de funciones de orden superior. `LEFT ARRAY JOIN`. Distribuido `GROUP BY`. Funciones que devuelven `Array`. Ejecución de `ORDER BY`. Escribir a `Distributed` tablas (nicelulu). Compatibilidad con versiones anteriores para `INSERT` consultas de clientes antiguos que implementan el `Native` protocolo. Soporte para `LowCardinality` para `JOIN`. Rendimiento mejorado cuando se trabaja en un solo flujo. [\#3823](https://github.com/ClickHouse/ClickHouse/pull/3823) [\#3803](https://github.com/ClickHouse/ClickHouse/pull/3803) [\#3799](https://github.com/ClickHouse/ClickHouse/pull/3799) [\#3769](https://github.com/ClickHouse/ClickHouse/pull/3769) [\#3744](https://github.com/ClickHouse/ClickHouse/pull/3744) [\#3681](https://github.com/ClickHouse/ClickHouse/pull/3681) [\#3651](https://github.com/ClickHouse/ClickHouse/pull/3651) [\#3649](https://github.com/ClickHouse/ClickHouse/pull/3649) [\#3641](https://github.com/ClickHouse/ClickHouse/pull/3641) [\#3632](https://github.com/ClickHouse/ClickHouse/pull/3632) [\#3568](https://github.com/ClickHouse/ClickHouse/pull/3568) [\#3523](https://github.com/ClickHouse/ClickHouse/pull/3523) [\#3518](https://github.com/ClickHouse/ClickHouse/pull/3518) +- Corregido cómo el `select_sequential_consistency` opción funciona. Anteriormente, cuando se habilitaba esta configuración, a veces se devolvía un resultado incompleto después de comenzar a escribir en una nueva partición. [\#2863](https://github.com/ClickHouse/ClickHouse/pull/2863) +- Las bases de datos se especifican correctamente al ejecutar DDL `ON CLUSTER` consultas y `ALTER UPDATE/DELETE`. [\#3772](https://github.com/ClickHouse/ClickHouse/pull/3772) [\#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) +- Las bases de datos se especifican correctamente para las subconsultas dentro de una VIEW. [\#3521](https://github.com/ClickHouse/ClickHouse/pull/3521) +- Se ha corregido un error en `PREWHERE` con `FINAL` para `VersionedCollapsingMergeTree`. [7167bfd7](https://github.com/ClickHouse/ClickHouse/commit/7167bfd7b365538f7a91c4307ad77e552ab4e8c1) +- Ahora puedes usar `KILL QUERY` para cancelar consultas que aún no se han iniciado porque están esperando a que se bloquee la tabla. [\#3517](https://github.com/ClickHouse/ClickHouse/pull/3517) +- Se corrigieron los cálculos de fecha y hora si los relojes se movían hacia atrás a la medianoche (esto sucede en Irán, y sucedió en Moscú de 1981 a 1983). Anteriormente, esto llevaba a que la hora se restableciera un día antes de lo necesario, y también causaba un formato incorrecto de la fecha y la hora en formato de texto. [\#3819](https://github.com/ClickHouse/ClickHouse/pull/3819) +- Se corrigieron errores en algunos casos de `VIEW` y subconsultas que omiten la base de datos. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3521) +- Se corrigió una condición de carrera al leer simultáneamente desde un `MATERIALIZED VIEW` y eliminar un `MATERIALIZED VIEW` debido a no bloquear el interior `MATERIALIZED VIEW`. [\#3404](https://github.com/ClickHouse/ClickHouse/pull/3404) [\#3694](https://github.com/ClickHouse/ClickHouse/pull/3694) +- Corregido el error `Lock handler cannot be nullptr.` [\#3689](https://github.com/ClickHouse/ClickHouse/pull/3689) +- Procesamiento de consultas fijo cuando el `compile_expressions` la opción está habilitada (está habilitada por defecto). Expresiones constantes no deterministas como el `now` función ya no están desplegados. [\#3457](https://github.com/ClickHouse/ClickHouse/pull/3457) +- Se corrigió un bloqueo al especificar un argumento de escala no constante en `toDecimal32/64/128` función. +- Se corrigió un error al intentar insertar una matriz con `NULL` elementos en el `Values` formato en una columna de tipo `Array` sin `Nullable` (si `input_format_values_interpret_expressions` = 1). [\#3487](https://github.com/ClickHouse/ClickHouse/pull/3487) [\#3503](https://github.com/ClickHouse/ClickHouse/pull/3503) +- Se corrigió el registro continuo de errores en `DDLWorker` si ZooKeeper no está disponible. [8f50c620](https://github.com/ClickHouse/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) +- Se corrigió el tipo de retorno para `quantile*` funciones de `Date` y `DateTime` de argumentos. [\#3580](https://github.com/ClickHouse/ClickHouse/pull/3580) +- Se corrigió el `WITH` cláusula si especifica un alias simple sin expresiones. [\#3570](https://github.com/ClickHouse/ClickHouse/pull/3570) +- Se corrigió el procesamiento de consultas con subconsultas con nombre y nombres de columna calificados cuando `enable_optimize_predicate_expression` está habilitado. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3588) +- Corregido el error `Attempt to attach to nullptr thread group` cuando se trabaja con vistas materializadas. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3623) +- Se corrigió un bloqueo al pasar ciertos argumentos incorrectos a la `arrayReverse` función. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +- Se corrigió el desbordamiento del búfer en el `extractURLParameter` función. Rendimiento mejorado. Se agregó el procesamiento correcto de cadenas que contienen cero bytes. [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/commit/141e9799e49201d84ea8e951d1bed4fb6d3dacb5) +- Desbordamiento de búfer fijo en el `lowerUTF8` y `upperUTF8` función. Se eliminó la capacidad de ejecutar estas funciones sobre `FixedString` argumentos de tipo. [\#3662](https://github.com/ClickHouse/ClickHouse/pull/3662) +- Se corrigió una rara condición de carrera al eliminar `MergeTree` tabla. [\#3680](https://github.com/ClickHouse/ClickHouse/pull/3680) +- Se corrigió una condición de carrera al leer desde `Buffer` tablas y realizar simultáneamente `ALTER` o `DROP` en las tablas de destino. [\#3719](https://github.com/ClickHouse/ClickHouse/pull/3719) +- Se corrigió una segfault si el `max_temporary_non_const_columns` se excedió el límite. [\#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) + +#### Mejora: {#improvements-1} + +- El servidor no escribe los archivos de configuración procesados `/etc/clickhouse-server/` directorio. En su lugar, los guarda en el `preprocessed_configs` directorio dentro `path`. Esto significa que el `/etc/clickhouse-server/` directorio no tiene acceso de escritura para el `clickhouse` usuario, lo que mejora la seguridad. [\#2443](https://github.com/ClickHouse/ClickHouse/pull/2443) +- El `min_merge_bytes_to_use_direct_io` La opción está establecida en 10 GiB de forma predeterminada. Una combinación que forma partes grandes de tablas de la familia MergeTree se realizará en `O_DIRECT` modo, que evita el desalojo excesivo de la caché de la página. [\#3504](https://github.com/ClickHouse/ClickHouse/pull/3504) +- Se inicia el servidor acelerado cuando hay un gran número de tablas. [\#3398](https://github.com/ClickHouse/ClickHouse/pull/3398) +- Se agregó un grupo de conexiones y HTTP `Keep-Alive` para conexiones entre réplicas. [\#3594](https://github.com/ClickHouse/ClickHouse/pull/3594) +- Si la sintaxis de la consulta no es válida, `400 Bad Request` el código se devuelve en el `HTTP` interfaz (500 se devolvió anteriormente). [31bc680a](https://github.com/ClickHouse/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) +- El `join_default_strictness` opción se establece en `ALL` por defecto para la compatibilidad. [120e2cbe](https://github.com/ClickHouse/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) +- Se eliminó el registro a `stderr` de la `re2` biblioteca para expresiones regulares no válidas o complejas. [\#3723](https://github.com/ClickHouse/ClickHouse/pull/3723) +- Añadido para el `Kafka` motor de tabla: comprueba si hay suscripciones antes de comenzar a leer de Kafka; la configuración kafka\_max\_block\_size para la tabla. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3396) +- El `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, y `murmurHash3_64` funciones ahora funcionan para cualquier número de argumentos y para argumentos en forma de tuplas. [\#3451](https://github.com/ClickHouse/ClickHouse/pull/3451) [\#3519](https://github.com/ClickHouse/ClickHouse/pull/3519) +- El `arrayReverse` función ahora funciona con cualquier tipo de matrices. [73e3a7b6](https://github.com/ClickHouse/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +- Se ha añadido un parámetro opcional: el tamaño de la ranura `timeSlots` función. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3724) +- Para `FULL` y `RIGHT JOIN`, el `max_block_size` configuración se utiliza para un flujo de datos no unidos de la tabla de la derecha. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3699) +- Se agregó el `--secure` parámetro de línea de comandos en `clickhouse-benchmark` y `clickhouse-performance-test` para habilitar TLS. [\#3688](https://github.com/ClickHouse/ClickHouse/pull/3688) [\#3690](https://github.com/ClickHouse/ClickHouse/pull/3690) +- Conversión de tipo cuando la estructura de un `Buffer` type table no coincide con la estructura de la tabla de destino. [Vitaly Baranov](https://github.com/ClickHouse/ClickHouse/pull/3603) +- Se agregó el `tcp_keep_alive_timeout` opción para habilitar los paquetes keep-alive después de la inactividad durante el intervalo de tiempo especificado. [\#3441](https://github.com/ClickHouse/ClickHouse/pull/3441) +- Eliminado cotización innecesaria de valores para la clave de partición en el `system.parts` tabla si consiste en una sola columna. [\#3652](https://github.com/ClickHouse/ClickHouse/pull/3652) +- La función de módulo funciona para `Date` y `DateTime` tipos de datos. [\#3385](https://github.com/ClickHouse/ClickHouse/pull/3385) +- Sinónimos añadidos para el `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, y `MID` función. [\#3774](https://github.com/ClickHouse/ClickHouse/pull/3774) [\#3763](https://github.com/ClickHouse/ClickHouse/pull/3763) Algunos nombres de funciones no distinguen entre mayúsculas y minúsculas por su compatibilidad con el estándar SQL. Azúcar sintáctico añadido `SUBSTRING(expr FROM start FOR length)` por compatibilidad con SQL. [\#3804](https://github.com/ClickHouse/ClickHouse/pull/3804) +- Se agregó la capacidad de `mlock` páginas de memoria correspondientes a `clickhouse-server` código ejecutable para evitar que se vea obligado a salir de la memoria. Esta función está deshabilitada de forma predeterminada. [\#3553](https://github.com/ClickHouse/ClickHouse/pull/3553) +- Rendimiento mejorado al leer desde `O_DIRECT` (con el `min_bytes_to_use_direct_io` opción activada). [\#3405](https://github.com/ClickHouse/ClickHouse/pull/3405) +- Rendimiento mejorado del `dictGet...OrDefault` función para un argumento de clave constante y un argumento predeterminado no constante. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3563) +- El `firstSignificantSubdomain` función ahora procesa los dominios `gov`, `mil`, y `edu`. [Igor Hatarist](https://github.com/ClickHouse/ClickHouse/pull/3601) Rendimiento mejorado. [\#3628](https://github.com/ClickHouse/ClickHouse/pull/3628) +- Posibilidad de especificar variables de entorno personalizadas para iniciar `clickhouse-server` utilizando el `SYS-V init.d` secuencia de comandos definiendo `CLICKHOUSE_PROGRAM_ENV` en `/etc/default/clickhouse`. + [Pavlo Bashynskyi](https://github.com/ClickHouse/ClickHouse/pull/3612) +- Corrija el código de retorno para el script init de clickhouse-server. [\#3516](https://github.com/ClickHouse/ClickHouse/pull/3516) +- El `system.metrics` ahora tiene el `VersionInteger` métrica, y `system.build_options` tiene la línea añadida `VERSION_INTEGER`, que contiene la forma numérica de la versión de ClickHouse, como `18016000`. [\#3644](https://github.com/ClickHouse/ClickHouse/pull/3644) +- Eliminado la capacidad de comparar el `Date` escriba con un número para evitar posibles errores como `date = 2018-12-17`, donde las comillas alrededor de la fecha se omiten por error. [\#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) +- Se corrigió el comportamiento de las funciones con estado como `rowNumberInAllBlocks`. Anteriormente generaron un resultado que era un número mayor debido al inicio durante el análisis de consultas. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3729) +- Si el `force_restore_data` no se puede eliminar, se muestra un mensaje de error. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3794) + +#### Mejoras de construcción: {#build-improvements-1} + +- Actualizado el `jemalloc` biblioteca, que corrige una posible pérdida de memoria. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3557) +- Perfilado con `jemalloc` está habilitado de forma predeterminada para depurar compilaciones. [2cc82f5c](https://github.com/ClickHouse/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) +- Se agregó la capacidad de ejecutar pruebas de integración cuando solo `Docker` está instalado en el sistema. [\#3650](https://github.com/ClickHouse/ClickHouse/pull/3650) +- Se agregó la prueba de expresión fuzz en consultas SELECT. [\#3442](https://github.com/ClickHouse/ClickHouse/pull/3442) +- Se agregó una prueba de esfuerzo para confirmaciones, que realiza pruebas funcionales en paralelo y en orden aleatorio para detectar más condiciones de carrera. [\#3438](https://github.com/ClickHouse/ClickHouse/pull/3438) +- Se mejoró el método para iniciar clickhouse-server en una imagen de Docker. [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/pull/3663) +- Para una imagen de Docker, se agregó soporte para inicializar bases de datos utilizando `/docker-entrypoint-initdb.d` directorio. [Konstantin Lebedev](https://github.com/ClickHouse/ClickHouse/pull/3695) +- Correcciones para construye en ARM. [\#3709](https://github.com/ClickHouse/ClickHouse/pull/3709) + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes} + +- Eliminado la capacidad de comparar el `Date` escriba con un número. En lugar de `toDate('2018-12-18') = 17883`, debe usar la conversión de tipo explícita `= toDate(17883)` [\#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) + +## Lanzamiento de ClickHouse 18.14 {#clickhouse-release-18-14} + +### Lanzamiento de ClickHouse 18.14.19, 2018-12-19 {#clickhouse-release-18-14-19-2018-12-19} + +#### Corrección de errores: {#bug-fixes-2} + +- Se corrigió un error que provocaba problemas con la actualización de diccionarios con el origen ODBC. [\#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [\#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) +- Las bases de datos se especifican correctamente al ejecutar DDL `ON CLUSTER` consulta. [\#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) +- Se corrigió una segfault si el `max_temporary_non_const_columns` se excedió el límite. [\#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) + +#### Mejoras de construcción: {#build-improvements-2} + +- Correcciones para construye en ARM. + +### Lanzamiento de ClickHouse 18.14.18, 2018-12-04 {#clickhouse-release-18-14-18-2018-12-04} + +#### Corrección de errores: {#bug-fixes-3} + +- Corregido el error en `dictGet...` función para diccionarios de tipo `range`, si uno de los argumentos es constante y otro no lo es. [\#3751](https://github.com/ClickHouse/ClickHouse/pull/3751) +- Se corrigió el error que causaba mensajes `netlink: '...': attribute type 1 has an invalid length` para imprimirse en el registro del kernel de Linux, eso estaba sucediendo solo en versiones suficientemente frescas del kernel de Linux. [\#3749](https://github.com/ClickHouse/ClickHouse/pull/3749) +- Segfault fijo en la función `empty` para argumento de `FixedString` tipo. [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/pull/3703) +- Se corrigió la asignación excesiva de memoria al usar un gran valor de `max_query_size` configuración (un fragmento de memoria de `max_query_size` bytes fue preasignado a la vez). [\#3720](https://github.com/ClickHouse/ClickHouse/pull/3720) + +#### Crear cambios: {#build-changes} + +- Compilación fija con las bibliotecas LLVM / Clang de la versión 7 de los paquetes del sistema operativo (estas bibliotecas se utilizan para la compilación de consultas en tiempo de ejecución). [\#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) + +### Lanzamiento de ClickHouse 18.14.17, 2018-11-30 {#clickhouse-release-18-14-17-2018-11-30} + +#### Corrección de errores: {#bug-fixes-4} + +- Se corrigieron casos en los que el proceso de puente ODBC no terminaba con el proceso del servidor principal. [\#3642](https://github.com/ClickHouse/ClickHouse/pull/3642) +- Inserción síncrona fija en el `Distributed` tabla con una lista de columnas que difiere de la lista de columnas de la tabla remota. [\#3673](https://github.com/ClickHouse/ClickHouse/pull/3673) +- Se corrigió una rara condición de carrera que podía provocar un bloqueo al soltar una tabla MergeTree. [\#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) +- Se corrigió un punto muerto de consulta en caso de que la creación del hilo de consulta fallara con el `Resource temporarily unavailable` error. [\#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) +- Análisis fijo del `ENGINE` cláusula cuando el `CREATE AS table` se utilizó la sintaxis y el `ENGINE` cláusula se especificó antes de la `AS table` (el error resultó en ignorar el motor especificado). [\#3692](https://github.com/ClickHouse/ClickHouse/pull/3692) + +### Lanzamiento de ClickHouse 18.14.15, 2018-11-21 {#clickhouse-release-18-14-15-2018-11-21} + +#### Corrección de errores: {#bug-fixes-5} + +- El tamaño del fragmento de memoria se sobreestimó al deserializar la columna de tipo `Array(String)` que conduce a “Memory limit exceeded” error. El problema apareció en la versión 18.12.13. [\#3589](https://github.com/ClickHouse/ClickHouse/issues/3589) + +### Lanzamiento de ClickHouse 18.14.14, 2018-11-20 {#clickhouse-release-18-14-14-2018-11-20} + +#### Corrección de errores: {#bug-fixes-6} + +- Fijo `ON CLUSTER` consultas cuando el clúster está configurado como seguro (indicador ``). [\#3599](https://github.com/ClickHouse/ClickHouse/pull/3599) + +#### Crear cambios: {#build-changes-1} + +- Problemas solucionados (llvm-7 del sistema, macos) [\#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) + +### Lanzamiento de ClickHouse 18.14.13, 2018-11-08 {#clickhouse-release-18-14-13-2018-11-08} + +#### Corrección de errores: {#bug-fixes-7} + +- Se corrigió el `Block structure mismatch in MergingSorted stream` error. [\#3162](https://github.com/ClickHouse/ClickHouse/issues/3162) +- Fijo `ON CLUSTER` consultas en caso de que se hayan activado las conexiones seguras en la configuración del clúster (el `` bandera). [\#3465](https://github.com/ClickHouse/ClickHouse/pull/3465) +- Se corrigió un error en las consultas que utilizaban `SAMPLE`, `PREWHERE` y columnas de alias. [\#3543](https://github.com/ClickHouse/ClickHouse/pull/3543) +- Se corrigió un raro `unknown compression method` error cuando el `min_bytes_to_use_direct_io` se habilitó la configuración. [3544](https://github.com/ClickHouse/ClickHouse/pull/3544) + +#### Mejoras de rendimiento: {#performance-improvements} + +- Regresión de rendimiento fija de consultas con `GROUP BY` de columnas de tipo UInt16 o Date cuando se ejecuta en procesadores AMD EPYC. [Igor Lapko](https://github.com/ClickHouse/ClickHouse/pull/3512) +- Regresión de rendimiento fija de las consultas que procesan cadenas largas. [\#3530](https://github.com/ClickHouse/ClickHouse/pull/3530) + +#### Mejoras de construcción: {#build-improvements-3} + +- Mejoras para simplificar la compilación de Arcadia. [\#3475](https://github.com/ClickHouse/ClickHouse/pull/3475), [\#3535](https://github.com/ClickHouse/ClickHouse/pull/3535) + +### Lanzamiento de ClickHouse 18.14.12, 2018-11-02 {#clickhouse-release-18-14-12-2018-11-02} + +#### Corrección de errores: {#bug-fixes-8} + +- Se corrigió un bloqueo al unir dos subconsultas sin nombre. [\#3505](https://github.com/ClickHouse/ClickHouse/pull/3505) +- Se corrigió la generación de consultas incorrectas (con un vacío `WHERE` cláusula) al consultar bases de datos externas. [hotid](https://github.com/ClickHouse/ClickHouse/pull/3477) +- Se corrigió usando un valor de tiempo de espera incorrecto en los diccionarios ODBC. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3511) + +### Lanzamiento de ClickHouse 18.14.11, 2018-10-29 {#clickhouse-release-18-14-11-2018-10-29} + +#### Corrección de errores: {#bug-fixes-9} + +- Corregido el error `Block structure mismatch in UNION stream: different number of columns` en consultas LIMIT. [\#2156](https://github.com/ClickHouse/ClickHouse/issues/2156) +- Se corrigieron errores al fusionar datos en tablas que contienen matrices dentro de estructuras anidadas. [\#3397](https://github.com/ClickHouse/ClickHouse/pull/3397) +- Se corrigieron los resultados de la consulta incorrecta si el `merge_tree_uniform_read_distribution` la configuración está deshabilitada (está habilitada por defecto). [\#3429](https://github.com/ClickHouse/ClickHouse/pull/3429) +- Se corrigió un error en las inserciones en una tabla distribuida en formato nativo. [\#3411](https://github.com/ClickHouse/ClickHouse/issues/3411) + +### Lanzamiento de ClickHouse 18.14.10, 2018-10-23 {#clickhouse-release-18-14-10-2018-10-23} + +- El `compile_expressions` configuración (compilación JIT de expresiones) está deshabilitada de forma predeterminada. [\#3410](https://github.com/ClickHouse/ClickHouse/pull/3410) +- El `enable_optimize_predicate_expression` configuración está deshabilitada de forma predeterminada. + +### Lanzamiento de ClickHouse 18.14.9, 2018-10-16 {#clickhouse-release-18-14-9-2018-10-16} + +#### Novedad: {#new-features-1} + +- El `WITH CUBE` modificador para `GROUP BY` (la sintaxis alternativa `GROUP BY CUBE(...)` también está disponible). [\#3172](https://github.com/ClickHouse/ClickHouse/pull/3172) +- Se agregó el `formatDateTime` función. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2770) +- Se agregó el `JDBC` motor de mesa y `jdbc` función de tabla (requiere la instalación de clickhouse-jdbc-bridge). [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) +- Funciones añadidas para trabajar con el número de semana ISO: `toISOWeek`, `toISOYear`, `toStartOfISOYear`, y `toDayOfYear`. [\#3146](https://github.com/ClickHouse/ClickHouse/pull/3146) +- Ahora puedes usar `Nullable` de columnas para `MySQL` y `ODBC` tabla. [\#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) +- Las estructuras de datos anidadas se pueden leer como objetos anidados en `JSONEachRow` formato. Se agregó el `input_format_import_nested_json` configuración. [Veloman Yunkan](https://github.com/ClickHouse/ClickHouse/pull/3144) +- El procesamiento paralelo está disponible para muchos `MATERIALIZED VIEW`s al insertar datos. Ver el `parallel_view_processing` configuración. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3208) +- Se agregó el `SYSTEM FLUSH LOGS` consulta (vacíos de registro forzados a tablas del sistema como `query_log`) [\#3321](https://github.com/ClickHouse/ClickHouse/pull/3321) +- Ahora puedes usar predefinidos `database` y `table` macros al declarar `Replicated` tabla. [\#3251](https://github.com/ClickHouse/ClickHouse/pull/3251) +- Se agregó la capacidad de leer `Decimal` valores de tipo en notación de ingeniería (indicando potencias de diez). [\#3153](https://github.com/ClickHouse/ClickHouse/pull/3153) + +#### Experimental características: {#experimental-features} + +- Optimización de la cláusula GROUP BY para `LowCardinality data types.` [\#3138](https://github.com/ClickHouse/ClickHouse/pull/3138) +- Cálculo optimizado de expresiones para `LowCardinality data types.` [\#3200](https://github.com/ClickHouse/ClickHouse/pull/3200) + +#### Mejora: {#improvements-2} + +- Consumo de memoria significativamente reducido para consultas con `ORDER BY` y `LIMIT`. Ver el `max_bytes_before_remerge_sort` configuración. [\#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) +- En ausencia de `JOIN` (`LEFT`, `INNER`, …), `INNER JOIN` se supone. [\#3147](https://github.com/ClickHouse/ClickHouse/pull/3147) +- Los asteriscos calificados funcionan correctamente en consultas con `JOIN`. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3202) +- El `ODBC` El motor de tabla elige correctamente el método para citar identificadores en el dialecto SQL de una base de datos remota. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/3210) +- El `compile_expressions` configuración (compilación JIT de expresiones) está habilitada de forma predeterminada. +- Comportamiento fijo para DROP DATABASE / TABLA SI EXISTE simultáneamente y CREATE DATABASE / TABLA SI NO EXISTE. Anteriormente, un `CREATE DATABASE ... IF NOT EXISTS` consulta podría devolver el mensaje de error “File … already exists” y el `CREATE TABLE ... IF NOT EXISTS` y `DROP TABLE IF EXISTS` las consultas podrían volver `Table ... is creating or attaching right now`. [\#3101](https://github.com/ClickHouse/ClickHouse/pull/3101) +- Las expresiones LIKE e IN con una mitad derecha constante se pasan al servidor remoto al realizar consultas desde tablas MySQL u ODBC. [\#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) +- Las comparaciones con expresiones constantes en una cláusula WHERE se pasan al servidor remoto al realizar consultas desde tablas MySQL y ODBC. Anteriormente, solo se pasaban las comparaciones con constantes. [\#3182](https://github.com/ClickHouse/ClickHouse/pull/3182) +- Cálculo correcto del ancho de fila en el terminal para `Pretty` formatos, incluyendo cadenas con jeroglíficos. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/3257). +- `ON CLUSTER` se puede especificar para `ALTER UPDATE` consulta. +- Rendimiento mejorado para leer datos en `JSONEachRow` formato. [\#3332](https://github.com/ClickHouse/ClickHouse/pull/3332) +- Sinónimos añadidos para el `LENGTH` y `CHARACTER_LENGTH` funciones de compatibilidad. El `CONCAT` la función ya no distingue entre mayúsculas y minúsculas. [\#3306](https://github.com/ClickHouse/ClickHouse/pull/3306) +- Se agregó el `TIMESTAMP` sinónimo de la `DateTime` tipo. [\#3390](https://github.com/ClickHouse/ClickHouse/pull/3390) +- Siempre hay espacio reservado para query\_id en los registros del servidor, incluso si la línea de registro no está relacionada con una consulta. Esto hace que sea más fácil analizar los registros de texto del servidor con herramientas de terceros. +- El consumo de memoria de una consulta se registra cuando supera el siguiente nivel de un número entero de gigabytes. [\#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) +- Se agregó el modo de compatibilidad para el caso cuando la biblioteca cliente que usa el protocolo nativo envía menos columnas por error de lo que el servidor espera para la consulta INSERT. Este escenario fue posible cuando se utiliza la biblioteca clickhouse-cpp. Anteriormente, este escenario hacía que el servidor se bloqueara. [\#3171](https://github.com/ClickHouse/ClickHouse/pull/3171) +- En una expresión WHERE definida por el usuario en `clickhouse-copier`, ahora puede usar un `partition_key` alias (para filtrado adicional por partición de tabla de origen). Esto es útil si el esquema de partición cambia durante la copia, pero solo cambia ligeramente. [\#3166](https://github.com/ClickHouse/ClickHouse/pull/3166) +- El flujo de trabajo del `Kafka` el motor se ha movido a un grupo de subprocesos en segundo plano para reducir automáticamente la velocidad de lectura de datos a altas cargas. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). +- Soporte para lectura `Tuple` y `Nested` de estructuras como `struct` en el `Cap'n'Proto format`. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3216) +- La lista de dominios de nivel superior `firstSignificantSubdomain` función ahora incluye el dominio `biz`. [disminución](https://github.com/ClickHouse/ClickHouse/pull/3219) +- En la configuración de diccionarios externos, `null_value` se interpreta como el valor del tipo de datos predeterminado. [\#3330](https://github.com/ClickHouse/ClickHouse/pull/3330) +- Soporte para el `intDiv` y `intDivOrZero` funciones para `Decimal`. [b48402e8](https://github.com/ClickHouse/ClickHouse/commit/b48402e8712e2b9b151e0eef8193811d433a1264) +- Soporte para el `Date`, `DateTime`, `UUID`, y `Decimal` tipos como una clave para el `sumMap` función de agregado. [\#3281](https://github.com/ClickHouse/ClickHouse/pull/3281) +- Soporte para el `Decimal` tipo de datos en diccionarios externos. [\#3324](https://github.com/ClickHouse/ClickHouse/pull/3324) +- Soporte para el `Decimal` tipo de datos en `SummingMergeTree` tabla. [\#3348](https://github.com/ClickHouse/ClickHouse/pull/3348) +- Añadido especializaciones para `UUID` en `if`. [\#3366](https://github.com/ClickHouse/ClickHouse/pull/3366) +- Reducido el número de `open` y `close` llamadas al sistema al leer desde un `MergeTree table`. [\#3283](https://github.com/ClickHouse/ClickHouse/pull/3283) +- A `TRUNCATE TABLE` consulta se puede ejecutar en cualquier réplica (la consulta se pasa a la réplica líder). [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3375) + +#### Corrección de errores: {#bug-fixes-10} + +- Se corrigió un problema con `Dictionary` tablas para `range_hashed` diccionario. Este error se produjo en la versión 18.12.17. [\#1702](https://github.com/ClickHouse/ClickHouse/pull/1702) +- Se corrigió un error al cargar `range_hashed` diccionarios (el mensaje `Unsupported type Nullable (...)`). Este error se produjo en la versión 18.12.17. [\#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) +- Se corrigieron errores en el `pointInPolygon` función debido a la acumulación de cálculos inexactos para polígonos con un gran número de vértices ubicados cerca uno del otro. [\#3331](https://github.com/ClickHouse/ClickHouse/pull/3331) [\#3341](https://github.com/ClickHouse/ClickHouse/pull/3341) +- Si después de fusionar partes de datos, la suma de comprobación de la parte resultante difiere del resultado de la misma fusión en otra réplica, el resultado de la fusión se elimina y la parte de datos se descarga de la otra réplica (este es el comportamiento correcto). Pero después de descargar la parte de datos, no se pudo agregar al conjunto de trabajo debido a un error de que la parte ya existe (porque la parte de datos se eliminó con cierto retraso después de la fusión). Esto llevó a intentos cíclicos de descargar los mismos datos. [\#3194](https://github.com/ClickHouse/ClickHouse/pull/3194) +- Se corrigió el cálculo incorrecto del consumo total de memoria por consultas (debido a un cálculo incorrecto, el `max_memory_usage_for_all_queries` la configuración funcionó incorrectamente y el `MemoryTracking` métrica tenía un valor incorrecto). Este error se produjo en la versión 18.12.13. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3344) +- Se corrigió la funcionalidad de `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` Este error se produjo en la versión 18.12.13. [\#3247](https://github.com/ClickHouse/ClickHouse/pull/3247) +- Se corrigió la preparación innecesaria de estructuras de datos para `JOIN`s en el servidor que inicia la consulta si `JOIN` sólo se realiza en servidores remotos. [\#3340](https://github.com/ClickHouse/ClickHouse/pull/3340) +- Se corrigieron errores en el `Kafka` engine: interbloqueos después de las excepciones al comenzar a leer datos, y se bloquea al finalizar [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3215). +- Para `Kafka` mesas, el opcional `schema` no se aprobó el parámetro (el esquema del `Cap'n'Proto` formato). [Vojtech Splichal](https://github.com/ClickHouse/ClickHouse/pull/3150) +- Si el conjunto de servidores ZooKeeper tiene servidores que aceptan la conexión pero luego la cierran inmediatamente en lugar de responder al protocolo de enlace, ClickHouse elige conectar otro servidor. Anteriormente, esto produjo el error `Cannot read all data. Bytes read: 0. Bytes expected: 4.` y el servidor no pudo iniciarse. [Nombre de la red inalámbrica (SSID):](https://github.com/ClickHouse/ClickHouse/commit/8218cf3a5f39a43401953769d6d12a0bb8d29da9) +- Si el conjunto de servidores ZooKeeper contiene servidores para los que la consulta DNS devuelve un error, estos servidores se ignoran. [17b8e209](https://github.com/ClickHouse/ClickHouse/commit/17b8e209221061325ad7ba0539f03c6e65f87f29) +- Conversión de tipo fijo entre `Date` y `DateTime` al insertar datos en el `VALUES` formato (si `input_format_values_interpret_expressions = 1`). Anteriormente, la conversión se realizaba entre el valor numérico del número de días en tiempo de Época Unix y la marca de tiempo Unix, lo que dio lugar a resultados inesperados. [\#3229](https://github.com/ClickHouse/ClickHouse/pull/3229) +- Conversión de tipo corregida entre `Decimal` y números enteros. [\#3211](https://github.com/ClickHouse/ClickHouse/pull/3211) +- Se corrigieron errores en el `enable_optimize_predicate_expression` configuración. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3231) +- Se ha corregido un error de análisis en formato CSV con números de coma flotante si se utiliza un separador CSV no predeterminado, como por ejemplo `;` [\#3155](https://github.com/ClickHouse/ClickHouse/pull/3155) +- Se corrigió el `arrayCumSumNonNegative` (no acumula valores negativos si el acumulador es menor que cero). [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/3163) +- Corregido cómo `Merge` las tablas funcionan encima de `Distributed` cuando se utiliza `PREWHERE`. [\#3165](https://github.com/ClickHouse/ClickHouse/pull/3165) +- Corrección de errores en el `ALTER UPDATE` consulta. +- Se corrigieron errores en el `odbc` función de tabla que apareció en la versión 18.12. [\#3197](https://github.com/ClickHouse/ClickHouse/pull/3197) +- Se corrigió el funcionamiento de las funciones agregadas con `StateArray` combinadores. [\#3188](https://github.com/ClickHouse/ClickHouse/pull/3188) +- Se corrigió un bloqueo al dividir un `Decimal` valor por cero. [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/commit/69dd6609193beb4e7acd3e6ad216eca0ccfb8179) +- Salida fija de tipos para operaciones usando `Decimal` y argumentos enteros. [\#3224](https://github.com/ClickHouse/ClickHouse/pull/3224) +- Corregido el segfault durante `GROUP BY` en `Decimal128`. [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/commit/3359ba06c39fcd05bfdb87d6c64154819621e13a) +- El `log_query_threads` configuración (información de registro sobre cada subproceso de ejecución de la consulta) ahora sólo tiene efecto si `log_queries` opción (información de registro sobre consultas) se establece en 1. Desde el `log_query_threads` la opción está habilitada de forma predeterminada, la información sobre los subprocesos se registró previamente incluso si el registro de consultas estaba deshabilitado. [\#3241](https://github.com/ClickHouse/ClickHouse/pull/3241) +- Se corrigió un error en la operación distribuida de la función de agregado de cuantiles (el mensaje de error `Not found column quantile...`). [Método de codificación de datos:](https://github.com/ClickHouse/ClickHouse/commit/292a885533b8e3b41ce8993867069d14cbd5a664) +- Se corrigió el problema de compatibilidad al trabajar en un clúster de servidores de la versión 18.12.17 y servidores anteriores al mismo tiempo. Para consultas distribuidas con claves GROUP BY de longitud fija y no fija, si había una gran cantidad de datos para agregar, los datos devueltos no siempre se agregaron completamente (dos filas diferentes contenían las mismas claves de agregación). [\#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) +- Manejo fijo de sustituciones en `clickhouse-performance-test`, si la consulta contiene sólo una parte de las sustituciones declaradas en la prueba. [\#3263](https://github.com/ClickHouse/ClickHouse/pull/3263) +- Se corrigió un error al usar `FINAL` con `PREWHERE`. [\#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) +- Se corrigió un error al usar `PREWHERE` sobre las columnas que se agregaron durante `ALTER`. [\#3298](https://github.com/ClickHouse/ClickHouse/pull/3298) +- Se agregó un cheque por la ausencia de `arrayJoin` para `DEFAULT` y `MATERIALIZED` expresiones. Previamente, `arrayJoin` llevado a un error al insertar datos. [\#3337](https://github.com/ClickHouse/ClickHouse/pull/3337) +- Se agregó un cheque por la ausencia de `arrayJoin` en una `PREWHERE` clausula. Anteriormente, esto llevó a mensajes como `Size ... doesn't match` o `Unknown compression method` al ejecutar consultas. [\#3357](https://github.com/ClickHouse/ClickHouse/pull/3357) +- Se corrigió el segfault que podía ocurrir en casos raros después de la optimización que reemplazaba las cadenas AND de las evaluaciones de igualdad con la expresión IN correspondiente. [Bienvenido a WordPress.](https://github.com/ClickHouse/ClickHouse/pull/3339) +- Correcciones menores a `clickhouse-benchmark`: anteriormente, la información del cliente no se enviaba al servidor; ahora el número de consultas ejecutadas se calcula con mayor precisión al apagar y para limitar el número de iteraciones. [\#3351](https://github.com/ClickHouse/ClickHouse/pull/3351) [\#3352](https://github.com/ClickHouse/ClickHouse/pull/3352) + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-1} + +- Eliminado el `allow_experimental_decimal_type` opcion. El `Decimal` tipo de datos está disponible para su uso predeterminado. [\#3329](https://github.com/ClickHouse/ClickHouse/pull/3329) + +## Lanzamiento de ClickHouse 18.12 {#clickhouse-release-18-12} + +### Lanzamiento de ClickHouse 18.12.17, 2018-09-16 {#clickhouse-release-18-12-17-2018-09-16} + +#### Novedad: {#new-features-2} + +- `invalidate_query` (la capacidad de especificar una consulta para comprobar si es necesario actualizar un diccionario externo) `clickhouse` fuente. [\#3126](https://github.com/ClickHouse/ClickHouse/pull/3126) +- Se agregó la capacidad de usar `UInt*`, `Int*`, y `DateTime` tipos de datos (junto con el `Date` tipo) como un `range_hashed` clave de diccionario externa que define los límites de los rangos. Ahora `NULL` se puede utilizar para designar un rango abierto. [Vasily Nemkov](https://github.com/ClickHouse/ClickHouse/pull/3123) +- El `Decimal` tipo ahora soporta `var*` y `stddev*` funciones agregadas. [\#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) +- El `Decimal` tipo ahora soporta funciones matemáticas (`exp`, `sin` y así sucesivamente.) [\#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) +- El `system.part_log` ahora tiene el `partition_id` columna. [\#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) + +#### Corrección de errores: {#bug-fixes-11} + +- `Merge` ahora funciona correctamente en `Distributed` tabla. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3159) +- Incompatibilidad fija (dependencia innecesaria del `glibc` versión) que hizo imposible ejecutar ClickHouse en `Ubuntu Precise` y versiones anteriores. La incompatibilidad surgió en la versión 18.12.13. [\#3130](https://github.com/ClickHouse/ClickHouse/pull/3130) +- Se corrigieron errores en el `enable_optimize_predicate_expression` configuración. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) +- Se corrigió un problema menor con compatibilidad con versiones anteriores que aparecía al trabajar con un clúster de réplicas en versiones anteriores a 18.12.13 y al mismo tiempo crear una nueva réplica de una tabla en un servidor con una versión más reciente (que se muestra en el mensaje `Can not clone replica, because the ... updated to new ClickHouse version`, lo cual es lógico, pero no debería suceder). [\#3122](https://github.com/ClickHouse/ClickHouse/pull/3122) + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-2} + +- El `enable_optimize_predicate_expression` opción está habilitada por defecto (que es bastante optimista). Si se producen errores de análisis de consultas relacionados con la búsqueda de los nombres de columna, establezca `enable_optimize_predicate_expression` a 0. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) + +### Lanzamiento de ClickHouse 18.12.14, 2018-09-13 {#clickhouse-release-18-12-14-2018-09-13} + +#### Novedad: {#new-features-3} + +- Añadido soporte para `ALTER UPDATE` consulta. [\#3035](https://github.com/ClickHouse/ClickHouse/pull/3035) +- Se agregó el `allow_ddl` opción, que restringe el acceso del usuario a consultas DDL. [\#3104](https://github.com/ClickHouse/ClickHouse/pull/3104) +- Se agregó el `min_merge_bytes_to_use_direct_io` opción para `MergeTree` los motores, que le permite establecer un umbral para el tamaño total de la fusión (cuando está por encima del umbral, los archivos de partes de datos se manejarán usando O\_DIRECT). [\#3117](https://github.com/ClickHouse/ClickHouse/pull/3117) +- El `system.merges` tabla del sistema ahora contiene el `partition_id` columna. [\#3099](https://github.com/ClickHouse/ClickHouse/pull/3099) + +#### Mejora {#improvements-3} + +- Si una parte de datos permanece sin cambios durante la mutación, no se descarga mediante réplicas. [\#3103](https://github.com/ClickHouse/ClickHouse/pull/3103) +- Autocompletar está disponible para nombres de configuraciones cuando se trabaja con `clickhouse-client`. [\#3106](https://github.com/ClickHouse/ClickHouse/pull/3106) + +#### Corrección de errores: {#bug-fixes-12} + +- Se agregó una verificación para los tamaños de las matrices que son elementos de `Nested` escriba campos al insertar. [\#3118](https://github.com/ClickHouse/ClickHouse/pull/3118) +- Se corrigió un error al actualizar los diccionarios externos con el `ODBC` fuente y `hashed` almacenamiento. Este error se produjo en la versión 18.12.13. +- Se corrigió un bloqueo al crear una tabla temporal a partir de una consulta con un `IN` condición. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3098) +- Se corrigió un error en las funciones agregadas para matrices que pueden tener `NULL` elemento. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/3097) + +### Lanzamiento de ClickHouse 18.12.13, 2018-09-10 {#clickhouse-release-18-12-13-2018-09-10} + +#### Novedad: {#new-features-4} + +- Se agregó el `DECIMAL(digits, scale)` tipo de datos (`Decimal32(scale)`, `Decimal64(scale)`, `Decimal128(scale)`). Para habilitarlo, use la configuración `allow_experimental_decimal_type`. [\#2846](https://github.com/ClickHouse/ClickHouse/pull/2846) [\#2970](https://github.com/ClickHouse/ClickHouse/pull/2970) [\#3008](https://github.com/ClickHouse/ClickHouse/pull/3008) [\#3047](https://github.com/ClickHouse/ClickHouse/pull/3047) +- Nuevo `WITH ROLLUP` modificador para `GROUP BY` (sintaxis alternativa: `GROUP BY ROLLUP(...)`). [\#2948](https://github.com/ClickHouse/ClickHouse/pull/2948) +- En consultas con JOIN, el carácter estrella se expande a una lista de columnas en todas las tablas, de acuerdo con el estándar SQL. Puede restaurar el comportamiento anterior configurando `asterisk_left_columns_only` a 1 en el nivel de configuración del usuario. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2787) +- Se agregó soporte para JOIN con funciones de tabla. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) +- Autocompletar presionando Tab en clickhouse-cliente. [Sergey Shcherbin](https://github.com/ClickHouse/ClickHouse/pull/2447) +- Ctrl + C en clickhouse-client borra una consulta que se ingresó. [\#2877](https://github.com/ClickHouse/ClickHouse/pull/2877) +- Se agregó el `join_default_strictness` ajuste (valores: `"`, `'any'`, `'all'`). Esto le permite no especificar `ANY` o `ALL` para `JOIN`. [\#2982](https://github.com/ClickHouse/ClickHouse/pull/2982) +- Cada línea del registro del servidor relacionada con el procesamiento de consultas muestra el ID de consulta. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Ahora puede obtener registros de ejecución de consultas en clickhouse-client (use el `send_logs_level` configuración). Con el procesamiento de consultas distribuidas, los registros se conectan en cascada desde todos los servidores. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- El `system.query_log` y `system.processes` (`SHOW PROCESSLIST`) las tablas ahora tienen información sobre todas las configuraciones modificadas al ejecutar una consulta (la estructura anidada `Settings` datos). Se agregó el `log_query_settings` configuración. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- El `system.query_log` y `system.processes` las tablas ahora muestran información sobre el número de subprocesos que participan en la ejecución de la consulta (consulte `thread_numbers` columna). [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Añadir `ProfileEvents` contadores que miden el tiempo dedicado a leer y escribir a través de la red y a leer y escribir en el disco, el número de errores de red y el tiempo dedicado a esperar cuando el ancho de banda de la red es limitado. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Añadir `ProfileEvents`contadores que contienen las métricas del sistema de rusage (puede usarlas para obtener información sobre el uso de CPU en el espacio de usuario y el kernel, errores de página y modificadores de contexto), así como métricas de taskstats (utilícelas para obtener información sobre el tiempo de espera de E / S, el tiempo de espera de CPU y la cantidad de datos leídos [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- El `ProfileEvents` los contadores se aplican globalmente y para cada consulta, así como para cada subproceso de ejecución de consulta, lo que le permite perfilar el consumo de recursos por consulta en detalle. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- Se agregó el `system.query_thread_log` tabla, que contiene información sobre cada subproceso de ejecución de consultas. Se agregó el `log_query_threads` configuración. [\#2482](https://github.com/ClickHouse/ClickHouse/pull/2482) +- El `system.metrics` y `system.events` las tablas ahora tienen documentación incorporada. [\#3016](https://github.com/ClickHouse/ClickHouse/pull/3016) +- Se agregó el `arrayEnumerateDense` función. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2975) +- Se agregó el `arrayCumSumNonNegative` y `arrayDifference` función. [Aleksey Studnev](https://github.com/ClickHouse/ClickHouse/pull/2942) +- Se agregó el `retention` función de agregado. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2887) +- Ahora puede agregar (fusionar) estados de funciones agregadas utilizando el operador más y multiplicar los estados de funciones agregadas por una constante no negativa. [\#3062](https://github.com/ClickHouse/ClickHouse/pull/3062) [\#3034](https://github.com/ClickHouse/ClickHouse/pull/3034) +- Las tablas de la familia MergeTree ahora tienen la columna virtual `_partition_id`. [\#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) + +#### Experimental características: {#experimental-features-1} + +- Se agregó el `LowCardinality(T)` tipo de datos. Este tipo de datos crea automáticamente un diccionario local de valores y permite el procesamiento de datos sin desempaquetar el diccionario. [\#2830](https://github.com/ClickHouse/ClickHouse/pull/2830) +- Se agregó un caché de funciones compiladas por JIT y un contador para el número de usos antes de compilar. Para compilar expresiones JIT, habilite la `compile_expressions` configuración. [\#2990](https://github.com/ClickHouse/ClickHouse/pull/2990) [\#3077](https://github.com/ClickHouse/ClickHouse/pull/3077) + +#### Mejora: {#improvements-4} + +- Se ha solucionado el problema con la acumulación ilimitada del registro de replicación cuando hay réplicas abandonadas. Se agregó un modo de recuperación efectivo para réplicas con un largo retraso. +- Rendimiento mejorado de `GROUP BY` con múltiples campos de agregación cuando uno de ellos es cadena y los otros son de longitud fija. +- Rendimiento mejorado al usar `PREWHERE` y con transferencia implícita de expresiones en `PREWHERE`. +- Rendimiento de análisis mejorado para formatos de texto (`CSV`, `TSV`). [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2977) [\#2980](https://github.com/ClickHouse/ClickHouse/pull/2980) +- Mejora del rendimiento de las cadenas de lectura y matrices en formatos binarios. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2955) +- Mayor rendimiento y menor consumo de memoria para consultas a `system.tables` y `system.columns` cuando hay un gran número de tablas en un solo servidor. [\#2953](https://github.com/ClickHouse/ClickHouse/pull/2953) +- Se corrigió un problema de rendimiento en el caso de una gran secuencia de consultas que resultaban en un error (el `_dl_addr` la función es visible en `perf top`, pero el servidor no está usando mucha CPU). [\#2938](https://github.com/ClickHouse/ClickHouse/pull/2938) +- Las condiciones se lanzan a la vista (cuando `enable_optimize_predicate_expression` está habilitado). [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2907) +- Mejoras en la funcionalidad del `UUID` tipo de datos. [\#3074](https://github.com/ClickHouse/ClickHouse/pull/3074) [\#2985](https://github.com/ClickHouse/ClickHouse/pull/2985) +- El `UUID` tipo de datos es compatible con los diccionarios de The-Alchemist. [\#2822](https://github.com/ClickHouse/ClickHouse/pull/2822) +- El `visitParamExtractRaw` funciona correctamente con estructuras anidadas. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2974) +- Cuando el `input_format_skip_unknown_fields` la configuración está habilitada, los campos de objeto `JSONEachRow` se omiten correctamente. [BlahGeek](https://github.com/ClickHouse/ClickHouse/pull/2958) +- Para un `CASE` expresión con condiciones, ahora puede omitir `ELSE`, que es equivalente a `ELSE NULL`. [\#2920](https://github.com/ClickHouse/ClickHouse/pull/2920) +- El tiempo de espera de la operación ahora se puede configurar cuando se trabaja con ZooKeeper. [urykhy](https://github.com/ClickHouse/ClickHouse/pull/2971) +- Puede especificar un desplazamiento para `LIMIT n, m` como `LIMIT n OFFSET m`. [\#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) +- Puede usar el `SELECT TOP n` sintaxis como una alternativa para `LIMIT`. [\#2840](https://github.com/ClickHouse/ClickHouse/pull/2840) +- Aumentó el tamaño de la cola para escribir en las tablas del sistema, por lo que `SystemLog parameter queue is full` el error no ocurre tan a menudo. +- El `windowFunnel` La función de agregado ahora admite eventos que cumplen múltiples condiciones. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2801) +- Las columnas duplicadas se pueden usar en un `USING` cláusula para `JOIN`. [\#3006](https://github.com/ClickHouse/ClickHouse/pull/3006) +- `Pretty` los formatos ahora tienen un límite en la alineación de columnas por ancho. Utilice el `output_format_pretty_max_column_pad_width` configuración. Si un valor es más amplio, aún se mostrará en su totalidad, pero las otras celdas de la tabla no serán demasiado anchas. [\#3003](https://github.com/ClickHouse/ClickHouse/pull/3003) +- El `odbc` función de tabla ahora le permite especificar el nombre de la base de datos / esquema. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2885) +- Se agregó la capacidad de usar un nombre de usuario especificado en el `clickhouse-client` archivo de configuración. [Vladimir Kozbin](https://github.com/ClickHouse/ClickHouse/pull/2909) +- El `ZooKeeperExceptions` se ha dividido en tres contadores: `ZooKeeperUserExceptions`, `ZooKeeperHardwareExceptions`, y `ZooKeeperOtherExceptions`. +- `ALTER DELETE` las consultas funcionan para vistas materializadas. +- Se agregó aleatorización al ejecutar el hilo de limpieza periódicamente para `ReplicatedMergeTree` para evitar picos de carga periódicos cuando hay un gran número de `ReplicatedMergeTree` tabla. +- Soporte para `ATTACH TABLE ... ON CLUSTER` consulta. [\#3025](https://github.com/ClickHouse/ClickHouse/pull/3025) + +#### Corrección de errores: {#bug-fixes-13} + +- Se corrigió un problema con `Dictionary` tablas (lanza el `Size of offsets doesn't match size of column` o `Unknown compression method` salvedad). Este error apareció en la versión 18.10.3. [\#2913](https://github.com/ClickHouse/ClickHouse/issues/2913) +- Se ha corregido un error al fusionar `CollapsingMergeTree` tablas si una de las partes de datos está vacía (estas partes se forman durante la fusión o `ALTER DELETE` si se han eliminado todos los datos), y `vertical` se utilizó el algoritmo para la fusión. [\#3049](https://github.com/ClickHouse/ClickHouse/pull/3049) +- Se corrigió una condición de carrera durante `DROP` o `TRUNCATE` para `Memory` tablas con una simultánea `SELECT`, lo que podría conducir a bloqueos del servidor. Este error apareció en la versión 1.1.54388. [\#3038](https://github.com/ClickHouse/ClickHouse/pull/3038) +- Se corrigió la posibilidad de pérdida de datos al insertar en `Replicated` tablas si el `Session is expired` se devuelve un error (la pérdida de datos puede ser detectada por el `ReplicatedDataLoss` métrica). Este error se produjo en la versión 1.1.54378. [\#2939](https://github.com/ClickHouse/ClickHouse/pull/2939) [\#2949](https://github.com/ClickHouse/ClickHouse/pull/2949) [\#2964](https://github.com/ClickHouse/ClickHouse/pull/2964) +- Se corrigió una falla seg durante `JOIN ... ON`. [\#3000](https://github.com/ClickHouse/ClickHouse/pull/3000) +- Se corrigió el error de búsqueda de nombres de columna cuando el `WHERE` la expresión consiste completamente en un nombre de columna completo, como `WHERE table.column`. [\#2994](https://github.com/ClickHouse/ClickHouse/pull/2994) +- Se corrigió el “Not found column” error que se produjo al ejecutar consultas distribuidas si se solicita una sola columna que consta de una expresión IN con una subconsulta desde un servidor remoto. [\#3087](https://github.com/ClickHouse/ClickHouse/pull/3087) +- Se corrigió el `Block structure mismatch in UNION stream: different number of columns` error que se produjo para las consultas distribuidas si uno de los fragmentos es local y el otro no, y la optimización del movimiento a `PREWHERE` se activa. [\#2226](https://github.com/ClickHouse/ClickHouse/pull/2226) [\#3037](https://github.com/ClickHouse/ClickHouse/pull/3037) [\#3055](https://github.com/ClickHouse/ClickHouse/pull/3055) [\#3065](https://github.com/ClickHouse/ClickHouse/pull/3065) [\#3073](https://github.com/ClickHouse/ClickHouse/pull/3073) [\#3090](https://github.com/ClickHouse/ClickHouse/pull/3090) [\#3093](https://github.com/ClickHouse/ClickHouse/pull/3093) +- Se corrigió el `pointInPolygon` función para ciertos casos de polígonos no convexos. [\#2910](https://github.com/ClickHouse/ClickHouse/pull/2910) +- Se corrigió el resultado incorrecto al comparar `nan` con enteros. [\#3024](https://github.com/ClickHouse/ClickHouse/pull/3024) +- Se corrigió un error en el `zlib-ng` biblioteca que podría conducir a segfault en casos raros. [\#2854](https://github.com/ClickHouse/ClickHouse/pull/2854) +- Se corrigió una pérdida de memoria al insertar en una tabla con `AggregateFunction` columnas, si el estado de la función agregada no es simple (asigna memoria por separado), y si una sola solicitud de inserción da como resultado múltiples bloques pequeños. [\#3084](https://github.com/ClickHouse/ClickHouse/pull/3084) +- Se corrigió una condición de carrera al crear y eliminar la misma `Buffer` o `MergeTree` mesa simultáneamente. +- Se corrigió la posibilidad de una segfault al comparar tuplas formadas por ciertos tipos no triviales, como tuplas. [\#2989](https://github.com/ClickHouse/ClickHouse/pull/2989) +- Se corrigió la posibilidad de un segfault al ejecutar ciertos `ON CLUSTER` consulta. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2960) +- Se corrigió un error en el `arrayDistinct` función para `Nullable` elementos de matriz. [\#2845](https://github.com/ClickHouse/ClickHouse/pull/2845) [\#2937](https://github.com/ClickHouse/ClickHouse/pull/2937) +- El `enable_optimize_predicate_expression` opción ahora soporta correctamente casos con `SELECT *`. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2929) +- Se corrigió el segfault al reinicializar la sesión de ZooKeeper. [\#2917](https://github.com/ClickHouse/ClickHouse/pull/2917) +- Se corrigió el bloqueo de potencial cuando se trabajaba con ZooKeeper. +- Se corrigió el código incorrecto para agregar estructuras de datos anidadas en un `SummingMergeTree`. +- Al asignar memoria para estados de funciones agregadas, la alineación se tiene en cuenta correctamente, lo que permite utilizar operaciones que requieren alineación al implementar estados de funciones agregadas. [Más información](https://github.com/ClickHouse/ClickHouse/pull/2808) + +#### Solución de seguridad: {#security-fix} + +- Uso seguro de las fuentes de datos ODBC. La interacción con los controladores ODBC utiliza un `clickhouse-odbc-bridge` proceso. Los errores en los controladores ODBC de terceros ya no causan problemas con la estabilidad del servidor o vulnerabilidades. [\#2828](https://github.com/ClickHouse/ClickHouse/pull/2828) [\#2879](https://github.com/ClickHouse/ClickHouse/pull/2879) [\#2886](https://github.com/ClickHouse/ClickHouse/pull/2886) [\#2893](https://github.com/ClickHouse/ClickHouse/pull/2893) [\#2921](https://github.com/ClickHouse/ClickHouse/pull/2921) +- Se corrigió la validación incorrecta de la ruta del archivo en el `catBoostPool` función de la tabla. [\#2894](https://github.com/ClickHouse/ClickHouse/pull/2894) +- El contenido de las tablas del sistema (`tables`, `databases`, `parts`, `columns`, `parts_columns`, `merges`, `mutations`, `replicas`, y `replication_queue`) se filtran de acuerdo con el acceso configurado por el usuario a las bases de datos (`allow_databases`). [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2856) + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-3} + +- En consultas con JOIN, el carácter estrella se expande a una lista de columnas en todas las tablas, de acuerdo con el estándar SQL. Puede restaurar el comportamiento anterior configurando `asterisk_left_columns_only` a 1 en el nivel de configuración del usuario. + +#### Crear cambios: {#build-changes-2} + +- La mayoría de las pruebas de integración ahora se pueden ejecutar mediante confirmación. +- Las comprobaciones de estilo de código también se pueden ejecutar mediante confirmación. +- El `memcpy` la implementación se elige correctamente cuando se construye en CentOS7 / Fedora. [Etienne Champetier](https://github.com/ClickHouse/ClickHouse/pull/2912) +- Al usar clang para compilar, algunas advertencias de `-Weverything` se han añadido, además de la `-Wall-Wextra -Werror`. [\#2957](https://github.com/ClickHouse/ClickHouse/pull/2957) +- La depuración de la compilación utiliza el `jemalloc` opción de depuración. +- La interfaz de la biblioteca para interactuar con ZooKeeper se declara abstracta. [\#2950](https://github.com/ClickHouse/ClickHouse/pull/2950) + +## Lanzamiento de ClickHouse 18.10 {#clickhouse-release-18-10} + +### Lanzamiento de ClickHouse 18.10.3, 2018-08-13 {#clickhouse-release-18-10-3-2018-08-13} + +#### Novedad: {#new-features-5} + +- HTTPS se puede utilizar para la replicación. [\#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) +- Se agregaron las funciones `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64`, y `murmurHash3_128` además de la existente `murmurHash2_32`. [\#2791](https://github.com/ClickHouse/ClickHouse/pull/2791) +- Compatibilidad con tipos Nullable en el controlador ODBC de ClickHouse (`ODBCDriver2` formato de salida). [\#2834](https://github.com/ClickHouse/ClickHouse/pull/2834) +- Soporte para `UUID` en las columnas clave. + +#### Mejora: {#improvements-5} + +- Los clústeres se pueden quitar sin reiniciar el servidor cuando se eliminan de los archivos de configuración. [\#2777](https://github.com/ClickHouse/ClickHouse/pull/2777) +- Los diccionarios externos se pueden quitar sin reiniciar el servidor cuando se eliminan de los archivos de configuración. [\#2779](https://github.com/ClickHouse/ClickHouse/pull/2779) +- Añadir `SETTINGS` soporte para el `Kafka` motor de mesa. [Alejandro Marshalov](https://github.com/ClickHouse/ClickHouse/pull/2781) +- Mejoras para el `UUID` tipo de datos (aún no completo). [\#2618](https://github.com/ClickHouse/ClickHouse/pull/2618) +- Soporte para piezas vacías después de fusiones en el `SummingMergeTree`, `CollapsingMergeTree` y `VersionedCollapsingMergeTree` motor. [\#2815](https://github.com/ClickHouse/ClickHouse/pull/2815) +- Se eliminan los registros antiguos de mutaciones completadas (`ALTER DELETE`). [\#2784](https://github.com/ClickHouse/ClickHouse/pull/2784) +- Se agregó el `system.merge_tree_settings` tabla. [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/2841) +- El `system.tables` la tabla ahora tiene columnas de dependencia: `dependencies_database` y `dependencies_table`. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2851) +- Se agregó el `max_partition_size_to_drop` opción de configuración. [\#2782](https://github.com/ClickHouse/ClickHouse/pull/2782) +- Se agregó el `output_format_json_escape_forward_slashes` opcion. [Alejandro Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2812) +- Se agregó el `max_fetch_partition_retries_count` configuración. [\#2831](https://github.com/ClickHouse/ClickHouse/pull/2831) +- Se agregó el `prefer_localhost_replica` configuración para deshabilitar la preferencia de una réplica local e ir a una réplica local sin interacción entre procesos. [\#2832](https://github.com/ClickHouse/ClickHouse/pull/2832) +- El `quantileExact` devoluciones de la función agregada `nan` en el caso de la agregación en un vacío `Float32` o `Float64` establecer. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2855) + +#### Corrección de errores: {#bug-fixes-14} + +- Se eliminó el escape innecesario de los parámetros de cadena de conexión para ODBC, lo que hizo imposible establecer una conexión. Este error se produjo en la versión 18.6.0. +- Se corrigió la lógica para el procesamiento `REPLACE PARTITION` comandos en la cola de replicación. Si hay dos `REPLACE` comandos para la misma partición, la lógica incorrecta podría hacer que uno de ellos permanezca en la cola de replicación y no se ejecute. [\#2814](https://github.com/ClickHouse/ClickHouse/pull/2814) +- Se corrigió un error de fusión cuando todas las partes de datos estaban vacías (partes que se formaron a partir de una fusión o de `ALTER DELETE` si se han eliminado todos los datos). Este error apareció en la versión 18.1.0. [\#2930](https://github.com/ClickHouse/ClickHouse/pull/2930) +- Se corrigió un error para concurrente `Set` o `Join`. [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2823) +- Se corrigió el `Block structure mismatch in UNION stream: different number of columns` error que ocurrió para `UNION ALL` consultas dentro de una subconsulta si una de las `SELECT` queries contiene nombres de columna duplicados. [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2094) +- Se corrigió una pérdida de memoria si se producía una excepción al conectarse a un servidor MySQL. +- Se corrigió el código de respuesta incorrecto de clickhouse-cliente en caso de un error de consulta. +- Se corrigió el comportamiento incorrecto de las vistas materializadas que contenían DISTINCT. [\#2795](https://github.com/ClickHouse/ClickHouse/issues/2795) + +#### Cambios incompatibles hacia atrás {#backward-incompatible-changes-4} + +- Se ha eliminado el soporte para consultas CHECK TABLE para tablas distribuidas. + +#### Crear cambios: {#build-changes-3} + +- El asignador ha sido reemplazado: `jemalloc` ahora se utiliza en lugar de `tcmalloc`. En algunos escenarios, esto aumenta la velocidad hasta un 20%. Sin embargo, hay consultas que se han ralentizado hasta en un 20%. El consumo de memoria se ha reducido en aproximadamente un 10% en algunos escenarios, con una estabilidad mejorada. Con cargas altamente competitivas, el uso de CPU en el espacio de usuario y en el sistema muestra solo un ligero aumento. [\#2773](https://github.com/ClickHouse/ClickHouse/pull/2773) +- Uso de libressl desde un submódulo. [\#1983](https://github.com/ClickHouse/ClickHouse/pull/1983) [\#2807](https://github.com/ClickHouse/ClickHouse/pull/2807) +- Uso de unixodbc desde un submódulo. [\#2789](https://github.com/ClickHouse/ClickHouse/pull/2789) +- Uso de mariadb-connector-c desde un submódulo. [\#2785](https://github.com/ClickHouse/ClickHouse/pull/2785) +- Se agregaron archivos de prueba funcionales al repositorio que dependen de la disponibilidad de los datos de prueba (por el momento, sin los datos de prueba en sí). + +## Lanzamiento de ClickHouse 18.6 {#clickhouse-release-18-6} + +### Lanzamiento de ClickHouse 18.6.0, 2018-08-02 {#clickhouse-release-18-6-0-2018-08-02} + +#### Novedad: {#new-features-6} + +- Se agregó soporte para expresiones ON para la sintaxis JOIN ON: + `JOIN ON Expr([table.]column ...) = Expr([table.]column, ...) [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]` + La expresión debe ser una cadena de igualdad unida por el operador AND. Cada lado de la igualdad puede ser una expresión arbitraria sobre las columnas de una de las tablas. Se admite el uso de nombres de columna completos (`table.name`, `database.table.name`, `table_alias.name`, `subquery_alias.name`) para la tabla correcta. [\#2742](https://github.com/ClickHouse/ClickHouse/pull/2742) +- HTTPS se puede habilitar para la replicación. [\#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) + +#### Mejora: {#improvements-6} + +- El servidor pasa el componente de parche de su versión al cliente. Los datos sobre el componente de la versión del parche se encuentran en `system.processes` y `query_log`. [\#2646](https://github.com/ClickHouse/ClickHouse/pull/2646) + +## Lanzamiento de ClickHouse 18.5 {#clickhouse-release-18-5} + +### Lanzamiento de ClickHouse 18.5.1, 2018-07-31 {#clickhouse-release-18-5-1-2018-07-31} + +#### Novedad: {#new-features-7} + +- Se agregó la función hash `murmurHash2_32` [\#2756](https://github.com/ClickHouse/ClickHouse/pull/2756). + +#### Mejora: {#improvements-7} + +- Ahora puedes usar el `from_env` [\#2741](https://github.com/ClickHouse/ClickHouse/pull/2741) atributo para establecer valores en archivos de configuración a partir de variables de entorno. +- Se agregaron versiones que no distinguen entre mayúsculas y minúsculas del `coalesce`, `ifNull`, y `nullIf functions` [\#2752](https://github.com/ClickHouse/ClickHouse/pull/2752). + +#### Corrección de errores: {#bug-fixes-15} + +- Se corrigió un posible error al iniciar una réplica [\#2759](https://github.com/ClickHouse/ClickHouse/pull/2759). + +## Lanzamiento de ClickHouse 18.4 {#clickhouse-release-18-4} + +### Lanzamiento de ClickHouse 18.4.0, 2018-07-28 {#clickhouse-release-18-4-0-2018-07-28} + +#### Novedad: {#new-features-8} + +- Tablas de sistema añadidas: `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [\#2721](https://github.com/ClickHouse/ClickHouse/pull/2721). +- Se agregó la capacidad de usar una función de tabla en lugar de una tabla como argumento de un `remote` o `cluster table function` [\#2708](https://github.com/ClickHouse/ClickHouse/pull/2708). +- Soporte para `HTTP Basic` autenticación en el protocolo de replicación [\#2727](https://github.com/ClickHouse/ClickHouse/pull/2727). +- El `has` función ahora permite buscar un valor numérico en una matriz de `Enum` valor [Maxim Khrisanfov](https://github.com/ClickHouse/ClickHouse/pull/2699). +- Soporte para agregar separadores de mensajes arbitrarios al leer desde `Kafka` [Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2701). + +#### Mejora: {#improvements-8} + +- El `ALTER TABLE t DELETE WHERE` La consulta no reescribe partes de datos que no se vieron afectadas por la condición WHERE [\#2694](https://github.com/ClickHouse/ClickHouse/pull/2694). +- El `use_minimalistic_checksums_in_zookeeper` opción para `ReplicatedMergeTree` tables está habilitada de forma predeterminada. Esta configuración se agregó en la versión 1.1.54378, 16-04-2018. Las versiones que son anteriores a 1.1.54378 ya no se pueden instalar. +- Soporte para correr `KILL` y `OPTIMIZE` consultas que especifican `ON CLUSTER` [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2689). + +#### Corrección de errores: {#bug-fixes-16} + +- Corregido el error `Column ... is not under an aggregate function and not in GROUP BY` para la agregación con una expresión IN. Este error apareció en la versión 18.1.0. ([bbdd780b](https://github.com/ClickHouse/ClickHouse/commit/bbdd780be0be06a0f336775941cdd536878dd2c2)) +- Se ha corregido un error en el `windowFunnel aggregate function` [Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2735). +- Se ha corregido un error en el `anyHeavy` función agregada ([a2101df2](https://github.com/ClickHouse/ClickHouse/commit/a2101df25a6a0fba99aa71f8793d762af2b801ee)) +- Se corrigió el bloqueo del servidor al usar el `countArray()` función de agregado. + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-5} + +- Parámetros para `Kafka` el motor fue cambiado de `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_schema, kafka_num_consumers])` a `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_row_delimiter, kafka_schema, kafka_num_consumers])`. Si sus tablas usan `kafka_schema` o `kafka_num_consumers` parámetros, debe editar manualmente los archivos de metadatos `path/metadata/database/table.sql` y añadir `kafka_row_delimiter` parámetro con `''` valor. + +## Lanzamiento de ClickHouse 18.1 {#clickhouse-release-18-1} + +### Lanzamiento de ClickHouse 18.1.0, 2018-07-23 {#clickhouse-release-18-1-0-2018-07-23} + +#### Novedad: {#new-features-9} + +- Soporte para el `ALTER TABLE t DELETE WHERE` consulta para tablas MergeTree no replicadas ([\#2634](https://github.com/ClickHouse/ClickHouse/pull/2634)). +- Soporte para tipos arbitrarios para el `uniq*` familia de funciones agregadas ([\#2010](https://github.com/ClickHouse/ClickHouse/issues/2010)). +- Soporte para tipos arbitrarios en operadores de comparación ([\#2026](https://github.com/ClickHouse/ClickHouse/issues/2026)). +- El `users.xml` archivo permite establecer una máscara de subred en el formato `10.0.0.1/255.255.255.0`. Esto es necesario para usar máscaras para redes IPv6 con ceros en el medio ([\#2637](https://github.com/ClickHouse/ClickHouse/pull/2637)). +- Se agregó el `arrayDistinct` función ([\#2670](https://github.com/ClickHouse/ClickHouse/pull/2670)). +- El motor SummingMergeTree ahora puede funcionar con columnas de tipo AggregateFunction ([Constantin S. Pan](https://github.com/ClickHouse/ClickHouse/pull/2566)). + +#### Mejora: {#improvements-9} + +- Se ha cambiado el esquema de numeración para las versiones de lanzamiento. Ahora la primera parte contiene el año de lanzamiento (A.D., zona horaria de Moscú, menos 2000), la segunda parte contiene el número de cambios importantes (aumentos para la mayoría de las versiones) y la tercera parte es la versión del parche. Las versiones siguen siendo compatibles con versiones anteriores, a menos que se indique lo contrario en el registro de cambios. +- Conversiones más rápidas de números de coma flotante a una cadena ([Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2664)). +- Si se omitieron algunas filas durante una inserción debido a errores de análisis (esto es posible con el `input_allow_errors_num` y `input_allow_errors_ratio` configuración activada), el número de filas omitidas ahora se escribe en el registro del servidor ([Leonardo Cecchi](https://github.com/ClickHouse/ClickHouse/pull/2669)). + +#### Corrección de errores: {#bug-fixes-17} + +- Se corrigió el comando TRUNCATE para tablas temporales ([Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2624)). +- Se corrigió un punto muerto raro en la biblioteca de cliente ZooKeeper que se producía cuando había un error de red al leer la respuesta ([c315200](https://github.com/ClickHouse/ClickHouse/commit/c315200e64b87e44bdf740707fc857d1fdf7e947)). +- Se corrigió un error durante un CAST a Nullable tipos ([\#1322](https://github.com/ClickHouse/ClickHouse/issues/1322)). +- Se corrigió el resultado incorrecto de la `maxIntersection()` función cuando los límites de los intervalos coincidieron ([Miguel Furmur](https://github.com/ClickHouse/ClickHouse/pull/2657)). +- Se corrigió la transformación incorrecta de la cadena de expresión OR en un argumento de función ([Más información](https://github.com/ClickHouse/ClickHouse/pull/2663)). +- Se corrigió la degradación del rendimiento para las consultas que contenían `IN (subquery)` expresiones dentro de otra subconsulta ([\#2571](https://github.com/ClickHouse/ClickHouse/issues/2571)). +- Se corrigió la incompatibilidad entre servidores con diferentes versiones en consultas distribuidas `CAST` función que no está en letras mayúsculas ([fe8c4d6](https://github.com/ClickHouse/ClickHouse/commit/fe8c4d64e434cacd4ceef34faa9005129f2190a5)). +- Se agregaron citas faltantes de identificadores para consultas a un DBMS externo ([\#2635](https://github.com/ClickHouse/ClickHouse/issues/2635)). + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-6} + +- La conversión de una cadena que contiene el número cero a DateTime no funciona. Ejemplo: `SELECT toDateTime('0')`. Esta es también la razón por la que `DateTime DEFAULT '0'` no funciona en tablas, así como `0` en los diccionarios. Solución: reemplazar `0` con `0000-00-00 00:00:00`. + +## Lanzamiento de ClickHouse 1.1 {#clickhouse-release-1-1} + +### Lanzamiento de ClickHouse 1.1.54394, 2018-07-12 {#clickhouse-release-1-1-54394-2018-07-12} + +#### Novedad: {#new-features-10} + +- Se agregó el `histogram` función agregada ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2521)). +- Ahora `OPTIMIZE TABLE ... FINAL` se puede utilizar sin especificar particiones para `ReplicatedMergeTree` ([Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2600)). + +#### Corrección de errores: {#bug-fixes-18} + +- Se corrigió un problema con un tiempo de espera muy pequeño para los sockets (un segundo) para leer y escribir al enviar y descargar datos replicados, lo que hacía imposible descargar partes más grandes si hay una carga en la red o el disco (resultó en intentos cíclicos para descargar partes). Este error se produjo en la versión 1.1.54388. +- Se corrigieron problemas al usar chroot en ZooKeeper si insertaba bloques de datos duplicados en la tabla. +- El `has` la función ahora funciona correctamente para una matriz con elementos Nullable ([\#2115](https://github.com/ClickHouse/ClickHouse/issues/2115)). +- El `system.tables` la tabla ahora funciona correctamente cuando se usa en consultas distribuidas. El `metadata_modification_time` y `engine_full` Las columnas ahora no son virtuales. Se corrigió un error que se producía si solo se consultaban estas columnas desde la tabla. +- Se corrigió cómo un vacío `TinyLog` la tabla funciona después de insertar un bloque de datos vacío ([\#2563](https://github.com/ClickHouse/ClickHouse/issues/2563)). +- El `system.zookeeper` table funciona si el valor del nodo en ZooKeeper es NULL. + +### Lanzamiento de ClickHouse 1.1.54390, 2018-07-06 {#clickhouse-release-1-1-54390-2018-07-06} + +#### Novedad: {#new-features-11} + +- Las consultas se pueden enviar en `multipart/form-data` formato (en el `query` campo), que es útil si también se envían datos externos para el procesamiento de consultas ([Olga Hvostikova](https://github.com/ClickHouse/ClickHouse/pull/2490)). +- Se agregó la capacidad de habilitar o deshabilitar el procesamiento de comillas simples o dobles al leer datos en formato CSV. Puede configurar esto en el `format_csv_allow_single_quotes` y `format_csv_allow_double_quotes` configuración ([Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2574)). +- Ahora `OPTIMIZE TABLE ... FINAL` se puede utilizar sin especificar la partición para variantes no replicadas de `MergeTree` ([Amos pájaro](https://github.com/ClickHouse/ClickHouse/pull/2599)). + +#### Mejora: {#improvements-10} + +- Rendimiento mejorado, consumo de memoria reducido y seguimiento correcto del consumo de memoria con el uso del operador IN cuando se podría usar un índice de tabla ([\#2584](https://github.com/ClickHouse/ClickHouse/pull/2584)). +- Se eliminó la comprobación redundante de las sumas de comprobación al agregar una parte de datos. Esto es importante cuando hay un gran número de réplicas, porque en estos casos el número total de comprobaciones fue igual a N^2. +- Añadido soporte para `Array(Tuple(...))` los argumentos para el `arrayEnumerateUniq` función ([\#2573](https://github.com/ClickHouse/ClickHouse/pull/2573)). +- Añadir `Nullable` soporte para el `runningDifference` función ([\#2594](https://github.com/ClickHouse/ClickHouse/pull/2594)). +- Mejora del rendimiento del análisis de consultas cuando hay un gran número de expresiones ([\#2572](https://github.com/ClickHouse/ClickHouse/pull/2572)). +- Selección más rápida de partes de datos para la fusión en `ReplicatedMergeTree` tabla. Recuperación más rápida de la sesión ZooKeeper ([\#2597](https://github.com/ClickHouse/ClickHouse/pull/2597)). +- El `format_version.txt` archivo para `MergeTree` tables se vuelve a crear si falta, lo que tiene sentido si ClickHouse se inicia después de copiar la estructura de directorios sin archivos ([Películas De Sexo](https://github.com/ClickHouse/ClickHouse/pull/2593)). + +#### Corrección de errores: {#bug-fixes-19} + +- Se corrigió un error al trabajar con ZooKeeper que podría hacer imposible recuperar la sesión y los estados de solo lectura de las tablas antes de reiniciar el servidor. +- Se ha corregido un error al trabajar con ZooKeeper que podría hacer que los nodos antiguos no se eliminen si se interrumpe la sesión. +- Se corrigió un error en el `quantileTDigest` función para argumentos Float (este error se introdujo en la versión 1.1.54388) ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2553)). +- Se corrigió un error en el índice de las tablas MergeTree si la columna de clave principal se encuentra dentro de la función para convertir tipos entre enteros con signo y sin signo del mismo tamaño ([\#2603](https://github.com/ClickHouse/ClickHouse/pull/2603)). +- Segfault fijo si `macros` se usan pero no están en el archivo de configuración ([\#2570](https://github.com/ClickHouse/ClickHouse/pull/2570)). +- Se corrigió el cambio a la base de datos predeterminada al volver a conectar el cliente ([\#2583](https://github.com/ClickHouse/ClickHouse/pull/2583)). +- Se corrigió un error que ocurría cuando el `use_index_for_in_with_subqueries` la configuración estaba deshabilitada. + +#### Solución de seguridad: {#security-fix-1} + +- El envío de archivos ya no es posible cuando se conecta a MySQL (`LOAD DATA LOCAL INFILE`). + +### Lanzamiento de ClickHouse 1.1.54388, 2018-06-28 {#clickhouse-release-1-1-54388-2018-06-28} + +#### Novedad: {#new-features-12} + +- Soporte para el `ALTER TABLE t DELETE WHERE` consulta para tablas replicadas. Se agregó el `system.mutations` para realizar un seguimiento del progreso de este tipo de consultas. +- Soporte para el `ALTER TABLE t [REPLACE|ATTACH] PARTITION` consulta para tablas \*MergeTree. +- Soporte para el `TRUNCATE TABLE` consulta ([Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2260)) +- Varios nuevos `SYSTEM` consultas para tablas replicadas (`RESTART REPLICAS`, `SYNC REPLICA`, `[STOP|START] [MERGES|FETCHES|SENDS REPLICATED|REPLICATION QUEUES]`). +- Se agregó la capacidad de escribir en una tabla con el motor MySQL y la función de tabla correspondiente ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2294)). +- Se agregó el `url()` función de la tabla y el `URL` motor de mesa ([Alejandro Sapin](https://github.com/ClickHouse/ClickHouse/pull/2501)). +- Se agregó el `windowFunnel` función agregada ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2352)). +- Nuevo `startsWith` y `endsWith` funciones para cadenas ([Vadim Plakhtinsky](https://github.com/ClickHouse/ClickHouse/pull/2429)). +- El `numbers()` función de tabla ahora le permite especificar el desplazamiento ([Invierno Zhang](https://github.com/ClickHouse/ClickHouse/pull/2535)). +- La contraseña para `clickhouse-client` se puede introducir de forma interactiva. +- Los registros del servidor ahora se pueden enviar a syslog ([Alejandro Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2459)). +- Compatibilidad con el inicio de sesión en diccionarios con una fuente de biblioteca compartida ([Alejandro Sapin](https://github.com/ClickHouse/ClickHouse/pull/2472)). +- Soporte para delimitadores CSV personalizados ([Ivan Zhukov](https://github.com/ClickHouse/ClickHouse/pull/2263)) +- Se agregó el `date_time_input_format` configuración. Si cambia esta configuración a `'best_effort'`, DateTime valores se leerán en una amplia gama de formatos. +- Se agregó el `clickhouse-obfuscator` utilidad para la ofuscación de datos. Ejemplo de uso: publicación de datos utilizados en pruebas de rendimiento. + +#### Experimental características: {#experimental-features-2} + +- Se agregó la capacidad de calcular `and` solo donde se necesitan ([Anastasia Tsarkova](https://github.com/ClickHouse/ClickHouse/pull/2272)) +- La compilación JIT a código nativo ya está disponible para algunas expresiones ([pyos](https://github.com/ClickHouse/ClickHouse/pull/2277)). + +#### Corrección de errores: {#bug-fixes-20} + +- Los duplicados ya no aparecen para una consulta con `DISTINCT` y `ORDER BY`. +- Consultas con `ARRAY JOIN` y `arrayFilter` ya no devuelve un resultado incorrecto. +- Se corrigió un error al leer una columna de matriz desde una estructura anidada ([\#2066](https://github.com/ClickHouse/ClickHouse/issues/2066)). +- Se corrigió un error al analizar consultas con una cláusula HAVING como `HAVING tuple IN (...)`. +- Se ha corregido un error al analizar consultas con alias recursivos. +- Se corrigió un error al leer desde ReplacingMergeTree con una condición en PREWHERE que filtra todas las filas ([\#2525](https://github.com/ClickHouse/ClickHouse/issues/2525)). +- La configuración del perfil de usuario no se aplicó al usar sesiones en la interfaz HTTP. +- Se corrigió cómo se aplican los ajustes desde los parámetros de línea de comandos en clickhouse-local. +- La biblioteca de cliente ZooKeeper ahora usa el tiempo de espera de sesión recibido del servidor. +- Se ha corregido un error en la biblioteca de cliente ZooKeeper cuando el cliente esperaba la respuesta del servidor más tiempo que el tiempo de espera. +- Poda fija de piezas para consultas con condiciones en columnas de clave de partición ([\#2342](https://github.com/ClickHouse/ClickHouse/issues/2342)). +- Las fusiones ahora son posibles después `CLEAR COLUMN IN PARTITION` ([\#2315](https://github.com/ClickHouse/ClickHouse/issues/2315)). +- Se ha corregido la asignación de tipos en la función de tabla ODBC ([sundy-li](https://github.com/ClickHouse/ClickHouse/pull/2268)). +- Las comparaciones de tipos se han fijado para `DateTime` con y sin la zona horaria ([Alejandro Bocharov](https://github.com/ClickHouse/ClickHouse/pull/2400)). +- Análisis sintáctico fijo y formato del `CAST` operador. +- Inserción fija en una vista materializada para el motor de tabla distribuida ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2411)). +- Se corrigió una condición de carrera al escribir datos desde el `Kafka` motor a vistas materializadas ([Información adicional](https://github.com/ClickHouse/ClickHouse/pull/2448)). +- SSRF fijo en la función de tabla remota (). +- Comportamiento de salida fijo de `clickhouse-client` en modo multilínea ([\#2510](https://github.com/ClickHouse/ClickHouse/issues/2510)). + +#### Mejora: {#improvements-11} + +- Las tareas en segundo plano de las tablas replicadas ahora se realizan en un grupo de subprocesos en lugar de en subprocesos separados ([Silviu Caragea](https://github.com/ClickHouse/ClickHouse/pull/1722)). +- Mejora del rendimiento de compresión LZ4. +- Análisis más rápido para consultas con un gran número de JOIN y subconsultas. +- La caché DNS ahora se actualiza automáticamente cuando hay demasiados errores de red. +- Las inserciones de tabla ya no se producen si la inserción en una de las vistas materializadas no es posible porque tiene demasiadas partes. +- Se ha corregido la discrepancia en los contadores de eventos `Query`, `SelectQuery`, y `InsertQuery`. +- Expresiones como `tuple IN (SELECT tuple)` se permiten si los tipos de tupla coinciden. +- Un servidor con tablas replicadas puede iniciarse incluso si no ha configurado ZooKeeper. +- Al calcular el número de núcleos de CPU disponibles, ahora se tienen en cuenta los límites en cgroups ([Más información](https://github.com/ClickHouse/ClickHouse/pull/2325)). +- Se agregó chown para los directorios de configuración en el archivo de configuración systemd ([Mikhail Shiryaev](https://github.com/ClickHouse/ClickHouse/pull/2421)). + +#### Crear cambios: {#build-changes-4} + +- El compilador gcc8 se puede usar para compilaciones. +- Se agregó la capacidad de construir llvm desde el submódulo. +- La versión de la biblioteca librdkafka se ha actualizado a v0.11.4. +- Se agregó la capacidad de usar la biblioteca libcpuid del sistema. La versión de la biblioteca se ha actualizado a 0.4.0. +- Se corrigió la compilación usando la biblioteca vectorclass ([Babacar Diassé](https://github.com/ClickHouse/ClickHouse/pull/2274)). +- Cmake ahora genera archivos para ninja de forma predeterminada (como cuando se usa `-G Ninja`). +- Se agregó la capacidad de usar la biblioteca libtinfo en lugar de libtermcap ([Nuestros Servicios](https://github.com/ClickHouse/ClickHouse/pull/2519)). +- Se corrigió un conflicto de archivos de encabezado en Fedora Rawhide ([\#2520](https://github.com/ClickHouse/ClickHouse/issues/2520)). + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-7} + +- Eliminado escapar en `Vertical` y `Pretty*` formatea y elimina el `VerticalRaw` formato. +- Si los servidores con la versión 1.1.54388 (o posterior) y los servidores con una versión anterior se utilizan simultáneamente en una consulta distribuida y la consulta tiene `cast(x, 'Type')` expresión sin `AS` palabra clave y no tiene la palabra `cast` en mayúsculas, se lanzará una excepción con un mensaje como `Not found column cast(0, 'UInt8') in block`. Solución: actualice el servidor en todo el clúster. + +### Lanzamiento de ClickHouse 1.1.54385, 2018-06-01 {#clickhouse-release-1-1-54385-2018-06-01} + +#### Corrección de errores: {#bug-fixes-21} + +- Se corrigió un error que en algunos casos causaba que las operaciones de ZooKeeper se bloquearan. + +### Lanzamiento de ClickHouse 1.1.54383, 2018-05-22 {#clickhouse-release-1-1-54383-2018-05-22} + +#### Corrección de errores: {#bug-fixes-22} + +- Se corrigió una desaceleración de la cola de replicación si una tabla tiene muchas réplicas. + +### Lanzamiento de ClickHouse 1.1.54381, 2018-05-14 {#clickhouse-release-1-1-54381-2018-05-14} + +#### Corrección de errores: {#bug-fixes-23} + +- Se corrigió una fuga de nodos en ZooKeeper cuando ClickHouse pierde la conexión con el servidor ZooKeeper. + +### Lanzamiento de ClickHouse 1.1.54380, 2018-04-21 {#clickhouse-release-1-1-54380-2018-04-21} + +#### Novedad: {#new-features-13} + +- Se agregó la función de tabla `file(path, format, structure)`. Un ejemplo de lectura de bytes de `/dev/urandom`: ``` ln -s /dev/urandom /var/lib/clickhouse/user_files/random``clickhouse-client -q "SELECT * FROM file('random', 'RowBinary', 'd UInt8') LIMIT 10" ```. + +#### Mejora: {#improvements-12} + +- Las subconsultas se pueden envolver en `()` para mejorar la legibilidad de las consultas. Por ejemplo: `(SELECT 1) UNION ALL (SELECT 1)`. +- Simple `SELECT` consultas de la `system.processes` no están incluidos en el `max_concurrent_queries` limite. + +#### Corrección de errores: {#bug-fixes-24} + +- Corregido el comportamiento incorrecto del `IN` operador cuando seleccione de `MATERIALIZED VIEW`. +- Se corrigió el filtrado incorrecto por índice de partición en expresiones como `partition_key_column IN (...)`. +- Se corrigió la imposibilidad de ejecutar `OPTIMIZE` consulta sobre réplica no líder si `REANAME` se realizó sobre la mesa. +- Se corrigió el error de autorización al ejecutar `OPTIMIZE` o `ALTER` consultas en una réplica que no sea de líder. +- Congelación fija de `KILL QUERY`. +- Se corrigió un error en la biblioteca de cliente ZooKeeper que provocaba la pérdida de observaciones, la congelación de la cola DDL distribuida y ralentizaciones en la cola de replicación si no estaba vacía `chroot` prefijo se utiliza en la configuración ZooKeeper. + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-8} + +- Se eliminó el soporte para expresiones como `(a, b) IN (SELECT (a, b))` (puede usar la expresión equivalente `(a, b) IN (SELECT a, b)`). En versiones anteriores, estas expresiones conducían a `WHERE` filtrado o errores causados. + +### Lanzamiento de ClickHouse 1.1.54378, 2018-04-16 {#clickhouse-release-1-1-54378-2018-04-16} + +#### Novedad: {#new-features-14} + +- El nivel de registro se puede cambiar sin reiniciar el servidor. +- Se agregó el `SHOW CREATE DATABASE` consulta. +- El `query_id` se puede pasar a `clickhouse-client` (elBroom). +- Nueva configuración: `max_network_bandwidth_for_all_users`. +- Añadido soporte para `ALTER TABLE ... PARTITION ...` para `MATERIALIZED VIEW`. +- Se agregó información sobre el tamaño de las partes de datos en forma sin comprimir en la tabla del sistema. +- Soporte de cifrado de servidor a servidor para tablas distribuidas (`1` en la configuración de réplica en ``). +- Configuración del nivel de tabla para el `ReplicatedMergeTree` familia con el fin de minimizar la cantidad de datos almacenados en Zookeeper: : `use_minimalistic_checksums_in_zookeeper = 1` +- Configuración del `clickhouse-client` pedir. De forma predeterminada, los nombres de servidor ahora se envían al mensaje. El nombre para mostrar del servidor se puede cambiar. También se envía en el `X-ClickHouse-Display-Name` Encabezado HTTP (Kirill Shvakov). +- Múltiples separados por comas `topics` se puede especificar para el `Kafka` motor (Tobias Adamson) +- Cuando una consulta es detenida por `KILL QUERY` o `replace_running_query` el cliente recibe el `Query was canceled` excepción en lugar de un resultado incompleto. + +#### Mejora: {#improvements-13} + +- `ALTER TABLE ... DROP/DETACH PARTITION` las consultas se ejecutan en la parte frontal de la cola de replicación. +- `SELECT ... FINAL` y `OPTIMIZE ... FINAL` se puede utilizar incluso cuando la tabla tiene una única parte de datos. +- A `query_log` se recrea sobre la marcha si se eliminó manualmente (Kirill Shvakov). +- El `lengthUTF8` funciona más rápido (zhang2014). +- Rendimiento mejorado de los insertos síncronos en `Distributed` tabla (`insert_distributed_sync = 1`) cuando hay una gran cantidad de fragmentos. +- El servidor acepta el `send_timeout` y `receive_timeout` configuraciones del cliente y las aplica cuando se conecta al cliente (se aplican en orden inverso: el socket del servidor `send_timeout` se establece en el `receive_timeout` valor recibido del cliente, y viceversa). +- Recuperación de bloqueos más robusta para la inserción asincrónica en `Distributed` tabla. +- El tipo de devolución del `countEqual` función cambiada de `UInt32` a `UInt64` (谢磊). + +#### Corrección de errores: {#bug-fixes-25} + +- Se corrigió un error con `IN` cuando el lado izquierdo de la expresión es `Nullable`. +- Ahora se devuelven los resultados correctos cuando se usan tuplas con `IN` cuando algunos de los componentes de la tupla están en el índice de la tabla. +- El `max_execution_time` limit ahora funciona correctamente con consultas distribuidas. +- Se corrigieron errores al calcular el tamaño de las columnas compuestas en el `system.columns` tabla. +- Se corrigió un error al crear una tabla temporal `CREATE TEMPORARY TABLE IF NOT EXISTS.` +- Corregidos errores en `StorageKafka` (\#\#2075) +- Se corrigieron fallos del servidor por argumentos no válidos de ciertas funciones agregadas. +- Se corrigió el error que impedía el `DETACH DATABASE` consulta de detener las tareas en segundo plano para `ReplicatedMergeTree` tabla. +- `Too many parts` es menos probable que ocurra al insertar en vistas materializadas agregadas (\#\# 2084). +- Se corrigió el manejo recursivo de sustituciones en la configuración si una sustitución debe ir seguida de otra sustitución en el mismo nivel. +- Se ha corregido la sintaxis en el archivo de metadatos al crear un `VIEW` que usa una consulta con `UNION ALL`. +- `SummingMergeTree` ahora funciona correctamente para la suma de estructuras de datos anidadas con una clave compuesta. +- Se corrigió la posibilidad de una condición de carrera al elegir el líder para `ReplicatedMergeTree` tabla. + +#### Crear cambios: {#build-changes-5} + +- La compilación admite `ninja` en lugar de `make` y usos `ninja` de forma predeterminada para la creación de versiones. +- Paquetes renombrados: `clickhouse-server-base` en `clickhouse-common-static`; `clickhouse-server-common` en `clickhouse-server`; `clickhouse-common-dbg` en `clickhouse-common-static-dbg`. Para instalar, utilice `clickhouse-server clickhouse-client`. Los paquetes con los nombres antiguos aún se cargarán en los repositorios por compatibilidad con versiones anteriores. + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-9} + +- Se eliminó la interpretación especial de una expresión IN si se especifica una matriz en el lado izquierdo. Anteriormente, la expresión `arr IN (set)` se interpretó como “at least one `arr` element belongs to the `set`”. Para obtener el mismo comportamiento en la nueva versión, escriba `arrayExists(x -> x IN (set), arr)`. +- Deshabilitado el uso incorrecto de la opción de socket `SO_REUSEPORT`, que se habilitó incorrectamente de forma predeterminada en la biblioteca Poco. Tenga en cuenta que en Linux ya no hay ninguna razón para especificar simultáneamente las direcciones `::` y `0.0.0.0` for listen – use just `::`, que permite escuchar la conexión tanto a través de IPv4 como IPv6 (con los ajustes de configuración predeterminados del kernel). También puede volver al comportamiento de versiones anteriores especificando `1` en la configuración. + +### Lanzamiento de ClickHouse 1.1.54370, 2018-03-16 {#clickhouse-release-1-1-54370-2018-03-16} + +#### Novedad: {#new-features-15} + +- Se agregó el `system.macros` tabla y actualización automática de macros cuando se cambia el archivo de configuración. +- Se agregó el `SYSTEM RELOAD CONFIG` consulta. +- Se agregó el `maxIntersections(left_col, right_col)` función de agregado, que devuelve el número máximo de intervalos de intersección simultáneamente `[left; right]`. El `maxIntersectionsPosition(left, right)` función devuelve el comienzo de la “maximum” intervalo. ([Miguel Furmur](https://github.com/ClickHouse/ClickHouse/pull/2012)). + +#### Mejora: {#improvements-14} + +- Al insertar datos en un `Replicated` se hacen menos solicitudes para `ZooKeeper` (y la mayoría de los errores de nivel de usuario han desaparecido del `ZooKeeper` registro). +- Se agregó la capacidad de crear alias para conjuntos de datos. Ejemplo: `WITH (1, 2, 3) AS set SELECT number IN set FROM system.numbers LIMIT 10`. + +#### Corrección de errores: {#bug-fixes-26} + +- Se corrigió el `Illegal PREWHERE` error al leer de las tablas Merge para `Distributed`tabla. +- Se agregaron correcciones que le permiten iniciar clickhouse-server en contenedores Docker solo para IPv4. +- Se corrigió una condición de carrera al leer desde el sistema `system.parts_columns tables.` +- Se eliminó el doble almacenamiento en búfer durante una inserción síncrona en un `Distributed` tabla, lo que podría haber causado que la conexión agotara el tiempo de espera. +- Se corrigió un error que causaba esperas excesivamente largas para una réplica no disponible antes de comenzar un `SELECT` consulta. +- Se corrigieron fechas incorrectas en el `system.parts` tabla. +- Se corrigió un error que hacía imposible insertar datos en un `Replicated` mesa si `chroot` no estaba vacío en la configuración del `ZooKeeper` Cluster. +- Se corrigió el algoritmo de fusión vertical para un vacío `ORDER BY` tabla. +- Se restauró la capacidad de usar diccionarios en consultas a tablas remotas, incluso si estos diccionarios no están presentes en el servidor del solicitante. Esta funcionalidad se perdió en la versión 1.1.54362. +- Restauró el comportamiento para consultas como `SELECT * FROM remote('server2', default.table) WHERE col IN (SELECT col2 FROM default.table)` cuando el lado derecho de la `IN` debe utilizar un control remoto `default.table` en lugar de uno local. Este comportamiento se rompió en la versión 1.1.54358. +- Se eliminó el registro de nivel de error extraño de `Not found column ... in block`. + +### ¿Qué puedes encontrar en Neodigit {#clickhouse-release-1-1-54362-2018-03-11} + +#### Novedad: {#new-features-16} + +- Agregación sin `GROUP BY` para un conjunto vacío (como `SELECT count(*) FROM table WHERE 0`) ahora devuelve un resultado con una fila con valores nulos para funciones agregadas, de acuerdo con el estándar SQL. Para restaurar el comportamiento anterior (devolver un resultado vacío), establezca `empty_result_for_aggregation_by_empty_set` a 1. +- Añadido tipo de conversión para `UNION ALL`. Se permiten diferentes nombres de alias en `SELECT` posiciones en `UNION ALL`, de acuerdo con el estándar SQL. +- Las expresiones arbitrarias se admiten en `LIMIT BY` clausula. Anteriormente, solo era posible usar columnas resultantes de `SELECT`. +- Un índice de `MergeTree` se utiliza cuando `IN` se aplica a una tupla de expresiones de las columnas de la clave principal. Ejemplo: `WHERE (UserID, EventDate) IN ((123, '2000-01-01'), ...)` (Anastasiya Tsarkova). +- Se agregó el `clickhouse-copier` herramienta para copiar entre clústeres y resharding datos (beta). +- Se agregaron funciones hash consistentes: `yandexConsistentHash`, `jumpConsistentHash`, `sumburConsistentHash`. Se pueden usar como una clave de fragmentación para reducir la cantidad de tráfico de red durante los subsiguientes reshardings. +- Funciones añadidas: `arrayAny`, `arrayAll`, `hasAny`, `hasAll`, `arrayIntersect`, `arrayResize`. +- Se agregó el `arrayCumSum` (Javi Santana). +- Se agregó el `parseDateTimeBestEffort`, `parseDateTimeBestEffortOrZero`, y `parseDateTimeBestEffortOrNull` funciones para leer el DateTime de una cadena que contiene texto en una amplia variedad de formatos posibles. +- Los datos se pueden volver a cargar parcialmente de diccionarios externos durante la actualización (cargar sólo los registros en los que el valor del campo especificado mayor que en la descarga anterior) (Arsen Hakobyan). +- Se agregó el `cluster` función de la tabla. Ejemplo: `cluster(cluster_name, db, table)`. El `remote` función de tabla puede aceptar el nombre del clúster como el primer argumento, si se especifica como un identificador. +- El `remote` y `cluster` las funciones de la tabla se pueden utilizar en `INSERT` consulta. +- Se agregó el `create_table_query` y `engine_full` columnas virtuales a la `system.tables`tabla . El `metadata_modification_time` columna es virtual. +- Se agregó el `data_path` y `metadata_path` columnas a `system.tables`y`system.databases` tablas, y añadió el `path` columna a la `system.parts` y `system.parts_columns` tabla. +- Se agregó información adicional sobre fusiones en el `system.part_log` tabla. +- Se puede utilizar una clave de partición arbitraria para `system.query_log` (Kirill Shvakov). +- El `SHOW TABLES` consulta ahora también muestra tablas temporales. Se agregaron tablas temporales y el `is_temporary` columna a `system.tables` (zhang2014). +- Añadir `DROP TEMPORARY TABLE` y `EXISTS TEMPORARY TABLE` consultas (zhang2014). +- Soporte para `SHOW CREATE TABLE` para tablas temporales (zhang2014). +- Se agregó el `system_profile` parámetro de configuración para los ajustes utilizados por los procesos internos. +- Soporte para carga `object_id` como un atributo en `MongoDB` diccionarios (Pavel Litvinenko). +- Lectura `null` como el valor predeterminado al cargar datos para un diccionario externo con el `MongoDB` fuente (Pavel Litvinenko). +- Lectura `DateTime` valores en el `Values` formato de una marca de tiempo Unix sin comillas simples. +- La conmutación por error se admite en `remote` funciones de tabla para los casos en que a algunas de las réplicas les falta la tabla solicitada. +- Los valores de configuración se pueden anular en la línea de comandos al ejecutar `clickhouse-server`. Ejemplo: `clickhouse-server -- --logger.level=information`. +- Implementado el `empty` función de un `FixedString` argumento: la función devuelve 1 si la cadena consta completamente de bytes nulos (zhang2014). +- Se agregó el `listen_try`parámetro de configuración para escuchar al menos una de las direcciones de escucha sin salir, si algunas de las direcciones no se pueden escuchar (útil para sistemas con soporte deshabilitado para IPv4 o IPv6). +- Se agregó el `VersionedCollapsingMergeTree` motor de mesa. +- Soporte para filas y tipos numéricos arbitrarios para el `library` fuente del diccionario. +- `MergeTree` las tablas se pueden usar sin una clave principal (debe especificar `ORDER BY tuple()`). +- A `Nullable` tipo puede ser `CAST` a un no-`Nullable` escriba si el argumento no es `NULL`. +- `RENAME TABLE` se puede realizar para `VIEW`. +- Se agregó el `throwIf` función. +- Se agregó el `odbc_default_field_size` opción, que le permite extender el tamaño máximo del valor cargado desde una fuente ODBC (por defecto, es 1024). +- El `system.processes` mesa y `SHOW PROCESSLIST` ahora tienen el `is_cancelled` y `peak_memory_usage` columna. + +#### Mejora: {#improvements-15} + +- Los límites y las cuotas sobre el resultado ya no se aplican a los datos intermedios para `INSERT SELECT` consultas o para `SELECT` subconsultas. +- Menos desencadenantes falsos de `force_restore_data` al comprobar el estado de `Replicated` cuando se inicia el servidor. +- Se agregó el `allow_distributed_ddl` opcion. +- Las funciones no deterministas no están permitidas en expresiones para `MergeTree` teclas de mesa. +- Archivos con sustituciones de `config.d` los directorios se cargan en orden alfabético. +- Rendimiento mejorado del `arrayElement` función en el caso de una matriz multidimensional constante con una matriz vacía como uno de los elementos. Ejemplo: `[[1], []][x]`. +- El servidor se inicia más rápido ahora cuando se utilizan archivos de configuración con sustituciones muy grandes (por ejemplo, listas muy grandes de redes IP). +- Al ejecutar una consulta, las funciones de valor de tabla se ejecutan una vez. Previamente, `remote` y `mysql` las funciones de valor de tabla realizaron la misma consulta dos veces para recuperar la estructura de tabla de un servidor remoto. +- El `MkDocs` se utiliza el generador de documentación. +- Cuando intenta eliminar una columna de tabla que `DEFAULT`/`MATERIALIZED` expresiones de otras columnas dependen, se lanza una excepción (zhang2014). +- Se agregó la capacidad de analizar una línea vacía en formatos de texto como el número 0 para `Float` tipos de datos. Esta característica estaba disponible anteriormente, pero se perdió en la versión 1.1.54342. +- `Enum` se pueden utilizar en `min`, `max`, `sum` y algunas otras funciones. En estos casos, utiliza los valores numéricos correspondientes. Esta característica estaba disponible anteriormente, pero se perdió en la versión 1.1.54337. +- Añadir `max_expanded_ast_elements` para restringir el tamaño del AST después de expandir recursivamente los alias. + +#### Corrección de errores: {#bug-fixes-27} + +- Fijo casos innecesarios columnas fueron retirados de las subconsultas en error, o no se quitan de subconsultas que contiene `UNION ALL`. +- Se corrigió un error en las fusiones para `ReplacingMergeTree` tabla. +- Inserciones síncronas fijas en `Distributed` tabla (`insert_distributed_sync = 1`). +- Segfault fijo para ciertos usos de `FULL` y `RIGHT JOIN` con columnas duplicadas en subconsultas. +- Segfault fijo para ciertos usos de `replace_running_query` y `KILL QUERY`. +- Se corrigió el orden de la `source` y `last_exception` columnas en el `system.dictionaries` tabla. +- Se corrigió un error cuando el `DROP DATABASE` la consulta no eliminó el archivo con metadatos. +- Se corrigió el `DROP DATABASE` consulta para `Dictionary` base. +- Se corrigió la baja precisión de `uniqHLL12` y `uniqCombined` funciones para cardinalidades superiores a 100 millones de artículos (Alex Bocharov). +- Se corrigió el cálculo de valores predeterminados implícitos cuando era necesario para calcular simultáneamente expresiones explícitas predeterminadas en `INSERT` consultas (zhang2014). +- Se corrigió un caso raro cuando una consulta a un `MergeTree` la tabla no pudo terminar (chenxing-xc). +- Se corrigió un bloqueo que se produjo al ejecutar un `CHECK` consulta para `Distributed` tablas si todos los fragmentos son locales (chenxing.xc). +- Se corrigió una ligera regresión de rendimiento con funciones que usan expresiones regulares. +- Se corrigió una regresión de rendimiento al crear matrices multidimensionales a partir de expresiones complejas. +- Se corrigió un error que podía causar un extra `FORMAT` sección para aparecer en una `.sql` archivo con metadatos. +- Se corrigió un error que causaba la `max_table_size_to_drop` límite para aplicar cuando se intenta eliminar un `MATERIALIZED VIEW` mirando una tabla explícitamente especificada. +- Se corrigió la incompatibilidad con clientes antiguos (a los clientes antiguos a veces se enviaban datos con el `DateTime('timezone')` tipo, que no entienden). +- Se ha corregido un error al leer `Nested` elementos de columna de estructuras que se agregaron usando `ALTER` pero que están vacíos para las particiones antiguas, cuando las condiciones para estas columnas se movieron a `PREWHERE`. +- Se corrigió un error al filtrar tablas por virtual `_table` columnas en consultas a `Merge` tabla. +- Se corrigió un error al usar `ALIAS` columnas en `Distributed` tabla. +- Se corrigió un error que hacía imposible la compilación dinámica para consultas con funciones agregadas del `quantile` familia. +- Se corrigió una condición de carrera en la canalización de ejecución de consultas que ocurría en casos muy raros al usar `Merge` con un gran número de tablas, y cuando se utiliza `GLOBAL` subconsultas. +- Se corrigió un bloqueo al pasar matrices de diferentes tamaños a un `arrayReduce` función cuando se utilizan funciones agregadas de múltiples argumentos. +- Prohibido el uso de consultas con `UNION ALL` en una `MATERIALIZED VIEW`. +- Se corrigió un error durante la inicialización del `part_log` tabla del sistema cuando se inicia el servidor (de forma predeterminada, `part_log` está deshabilitado). + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-10} + +- Eliminado el `distributed_ddl_allow_replicated_alter` opcion. Este comportamiento está habilitado de forma predeterminada. +- Eliminado el `strict_insert_defaults` configuración. Si estaba utilizando esta funcionalidad, escriba en `clickhouse-feedback@yandex-team.com`. +- Eliminado el `UnsortedMergeTree` motor. + +### ¿Qué puedes encontrar en Neodigit {#clickhouse-release-1-1-54343-2018-02-05} + +- Se agregó soporte de macros para definir nombres de clúster en consultas DDL distribuidas y constructores de tablas distribuidas: `CREATE TABLE distr ON CLUSTER '{cluster}' (...) ENGINE = Distributed('{cluster}', 'db', 'table')`. +- Ahora consultas como `SELECT ... FROM table WHERE expr IN (subquery)` se procesan utilizando el `table` Indice. +- Se ha mejorado el procesamiento de duplicados al insertar en tablas replicadas, por lo que ya no ralentizan la ejecución de la cola de replicación. + +### ¿Qué puedes encontrar en Neodigit {#clickhouse-release-1-1-54342-2018-01-22} + +Esta versión contiene correcciones de errores para la versión anterior 1.1.54337: + +- Se corrigió una regresión en 1.1.54337: si el usuario predeterminado tiene acceso de solo lectura, entonces el servidor se niega a iniciar con el mensaje `Cannot create database in readonly mode`. +- Se corrigió una regresión en 1.1.54337: en sistemas con systemd, los registros siempre se escriben en syslog independientemente de la configuración; el script de vigilancia todavía usa init .d. +- Se corrigió una regresión en 1.1.54337: configuración predeterminada incorrecta en la imagen de Docker. +- Comportamiento no determinista fijo de GraphiteMergeTree (se puede ver en los mensajes de registro `Data after merge is not byte-identical to the data on another replicas`). +- Se corrigió un error que podía provocar fusiones inconsistentes después de OPTIMIZE consulta a tablas replicadas (es posible que lo vea en los mensajes de registro `Part ... intersects the previous part`). +- Las tablas de búfer ahora funcionan correctamente cuando las columnas MATERIALIZED están presentes en la tabla de destino (por zhang2014). +- Se corrigió un error en la implementación de NULL. + +### ¿Qué puedes encontrar en Neodigit {#clickhouse-release-1-1-54337-2018-01-18} + +#### Novedad: {#new-features-17} + +- Se agregó soporte para el almacenamiento de matrices multidimensionales y tuplas (`Tuple` tipo de datos) en las tablas. +- Soporte para funciones de mesa para `DESCRIBE` y `INSERT` consulta. Se agregó soporte para subconsultas en `DESCRIBE`. Ejemplos: `DESC TABLE remote('host', default.hits)`; `DESC TABLE (SELECT 1)`; `INSERT INTO TABLE FUNCTION remote('host', default.hits)`. Soporte para `INSERT INTO TABLE` además de `INSERT INTO`. +- Soporte mejorado para zonas horarias. El `DateTime` tipo de datos se puede anotar con la zona horaria que se utiliza para el análisis y el formato en formatos de texto. Ejemplo: `DateTime('Europe/Moscow')`. Cuando las zonas horarias se especifican en `DateTime` argumentos, el tipo de devolución rastreará la zona horaria y el valor se mostrará como se esperaba. +- Se agregaron las funciones `toTimeZone`, `timeDiff`, `toQuarter`, `toRelativeQuarterNum`. El `toRelativeHour`/`Minute`/`Second` funciones pueden tomar un valor de tipo `Date` como argumento. El `now` nombre de la función distingue entre mayúsculas y minúsculas. +- Se agregó el `toStartOfFifteenMinutes` (Kirill Shvakov). +- Se agregó el `clickhouse format` herramienta para formatear consultas. +- Se agregó el `format_schema_path` configuration parameter (Marek Vavruşa). It is used for specifying a schema in `Cap'n Proto` formato. Los archivos de esquema solo se pueden ubicar en el directorio especificado. +- Se agregó soporte para sustituciones de configuración (`incl` y `conf.d`) para la configuración de diccionarios y modelos externos (Pavel Yakunin). +- Se agregó una columna con documentación para el `system.settings` (Kirill Shvakov). +- Se agregó el `system.parts_columns` tabla con información sobre los tamaños de columna en cada parte de datos `MergeTree` tabla. +- Se agregó el `system.models` tabla con información sobre `CatBoost` modelos de aprendizaje automático. +- Se agregó el `mysql` y `odbc` función de la tabla y correspondiente `MySQL` y `ODBC` para acceder a bases de datos remotas. Esta funcionalidad se encuentra en la fase beta. +- Se agregó la posibilidad de pasar un argumento de tipo `AggregateFunction` para el `groupArray` función agregada (para que pueda crear una matriz de estados de alguna función agregada). +- Se eliminaron restricciones en varias combinaciones de combinadores de funciones agregadas. Por ejemplo, puede usar `avgForEachIf` así como `avgIfForEach` funciones agregadas, que tienen diferentes comportamientos. +- El `-ForEach` el combinador de funciones agregadas se extiende para el caso de funciones agregadas de múltiples argumentos. +- Se agregó soporte para funciones agregadas de `Nullable` argumentos incluso para los casos en que la función devuelve un no-`Nullable` resultado (añadido con la contribución de Silviu Caragea). Ejemplo: `groupArray`, `groupUniqArray`, `topK`. +- Se agregó el `max_client_network_bandwidth` para `clickhouse-client` (Kirill Shvakov). +- Los usuarios con `readonly = 2` setting are allowed to work with TEMPORARY tables (CREATE, DROP, INSERT…) (Kirill Shvakov). +- Se agregó soporte para el uso de múltiples consumidores con el `Kafka` motor. Opciones de configuración ampliadas para `Kafka` (Marek Vavruša). +- Se agregó el `intExp3` y `intExp4` función. +- Se agregó el `sumKahan` función de agregado. +- Se agregaron las funciones \* Number\* OrNull, donde \* Number\* es un tipo numérico. +- Añadido soporte para `WITH` para una `INSERT SELECT` consulta (autor: zhang2014). +- Configuración añadida: `http_connection_timeout`, `http_send_timeout`, `http_receive_timeout`. En particular, estos valores se utilizan para descargar partes de datos para la replicación. Cambiar esta configuración permite una conmutación por error más rápida si la red está sobrecargada. +- Añadido soporte para `ALTER` para tablas de tipo `Null` (Anastasiya Tsarkova). +- El `reinterpretAsString` se extiende para todos los tipos de datos que se almacenan contiguamente en la memoria. +- Se agregó el `--silent` opción para el `clickhouse-local` herramienta. Suprime la información de ejecución de consultas de impresión en stderr. +- Se agregó soporte para leer valores de tipo `Date` de texto en un formato donde el mes y / o día del mes se especifica utilizando un solo dígito en lugar de dos dígitos (Amos Bird). + +#### Optimizaciones de rendimiento: {#performance-optimizations} + +- Rendimiento mejorado de las funciones agregadas `min`, `max`, `any`, `anyLast`, `anyHeavy`, `argMin`, `argMax` de argumentos de cadena. +- Mejora del rendimiento de las funciones `isInfinite`, `isFinite`, `isNaN`, `roundToExp2`. +- Rendimiento mejorado del análisis y el formato `Date` y `DateTime` valores de tipo en formato de texto. +- Mejora del rendimiento y la precisión del análisis de números de coma flotante. +- Menor uso de memoria para `JOIN` en el caso cuando las partes izquierda y derecha tienen columnas con nombres idénticos que no están contenidos en `USING` . +- Rendimiento mejorado de las funciones agregadas `varSamp`, `varPop`, `stddevSamp`, `stddevPop`, `covarSamp`, `covarPop`, `corr` reduciendo la estabilidad computacional. Las funciones antiguas están disponibles bajo los nombres `varSampStable`, `varPopStable`, `stddevSampStable`, `stddevPopStable`, `covarSampStable`, `covarPopStable`, `corrStable`. + +#### Corrección de errores: {#bug-fixes-28} + +- Deduplicación de datos fija después de ejecutar un `DROP` o `DETACH PARTITION` consulta. En la versión anterior, soltar una partición e insertar los mismos datos de nuevo no funcionaba porque los bloques insertados se consideraban duplicados. +- Se corrigió un error que podía conducir a una interpretación incorrecta de la `WHERE` cláusula para `CREATE MATERIALIZED VIEW` consultas con `POPULATE` . +- Se corrigió un error al usar el `root_path` parámetro en el `zookeeper_servers` configuración. +- Se corrigieron los resultados inesperados de pasar el `Date` argumento a `toStartOfDay` . +- Se corrigió el `addMonths` y `subtractMonths` funciones y la aritmética para `INTERVAL n MONTH` en los casos en que el resultado tiene el año anterior. +- Se agregó soporte faltante para el `UUID` tipo de datos para `DISTINCT` , `JOIN` , y `uniq` funciones agregadas y diccionarios externos (Evgeniy Ivanov). Soporte para `UUID` todavía está incompleto. +- Fijo `SummingMergeTree` comportamiento en los casos en que las filas suman a cero. +- Varias correcciones para el `Kafka` engine (Marek Vavruša). +- Corregido el comportamiento incorrecto del `Join` motor de mesa (Amos Bird). +- Se corrigió el comportamiento incorrecto del asignador en FreeBSD y OS X. +- El `extractAll` la función ahora admite coincidencias vacías. +- Se corrigió un error que bloqueaba el uso de `libressl` en lugar de `openssl` . +- Se corrigió el `CREATE TABLE AS SELECT` consulta de tablas temporales. +- Se corrigió la no anatomía de la actualización de la cola de replicación. Esto podría provocar que las réplicas no estén sincronizadas hasta que se reinicie el servidor. +- Posible desbordamiento fijo en `gcd` , `lcm` y `modulo` (`%` (Maks Skorokhod). +- `-preprocessed` los archivos ahora se crean después de cambiar `umask` (`umask` se puede cambiar en la configuración). +- Se corrigió un error en la verificación de antecedentes de las piezas (`MergeTreePartChecker` ) cuando se utiliza una clave de partición personalizada. +- Análisis fijo de tuplas (valores de la `Tuple` tipo de datos) en formatos de texto. +- Mensajes de error mejorados sobre tipos incompatibles pasados a `multiIf` , `array` y algunas otras funciones. +- Soporte rediseñado para `Nullable` tipo. Se corrigieron errores que podían provocar un bloqueo del servidor. Se corrigieron casi todos los demás errores relacionados con `NULL` soporte: conversiones de tipo incorrectas en INSERT SELECT, soporte insuficiente para Nullable en HAVING y PREWHERE, `join_use_nulls` modo, tipos anulables como argumentos de `OR` operador, etc. +- Se corrigieron varios errores relacionados con la semántica interna de los tipos de datos. Ejemplos: suma innecesaria de `Enum` campos de tipo en `SummingMergeTree` ; alineación de `Enum` tipos en `Pretty` formatos, etc. +- Comprueba más estrictamente las combinaciones permitidas de columnas compuestas. +- Se corrigió el desbordamiento al especificar un parámetro muy grande para el `FixedString` tipo de datos. +- Se ha corregido un error en el `topK` función de agregado en un caso genérico. +- Se agregó la verificación faltante de igualdad de tamaños de matriz en argumentos de variantes narias de funciones agregadas con un `-Array` combinador. +- Se ha corregido un error en `--pager` para `clickhouse-client` (autor: ks1322). +- Se corrigió la precisión del `exp10` función. +- Se corrigió el comportamiento del `visitParamExtract` función para un mejor cumplimiento de la documentación. +- Se corrigió el bloqueo cuando se especifican tipos de datos incorrectos. +- Se corrigió el comportamiento de `DISTINCT` en el caso cuando todas las columnas son constantes. +- Se corrigió el formato de consulta en el caso de usar el `tupleElement` función con una expresión constante compleja como el índice del elemento de tupla. +- Se ha corregido un error en `Dictionary` tablas para `range_hashed` diccionario. +- Se corrigió un error que provocaba filas excesivas en el resultado de `FULL` y `RIGHT JOIN` (Amós De Aves). +- Se corrigió un bloqueo del servidor al crear y eliminar archivos temporales en `config.d` directorios durante la recarga de configuración. +- Se corrigió el `SYSTEM DROP DNS CACHE` consulta: la memoria caché se vació pero las direcciones de los nodos del clúster no se actualizaron. +- Se corrigió el comportamiento de `MATERIALIZED VIEW` después de ejecutar `DETACH TABLE` for the table under the view (Marek Vavruša). + +#### Mejoras de construcción: {#build-improvements-4} + +- El `pbuilder` herramienta se utiliza para compilaciones. El proceso de compilación es casi completamente independiente del entorno de host de compilación. +- Una sola compilación se utiliza para diferentes versiones del sistema operativo. Los paquetes y binarios se han hecho compatibles con una amplia gama de sistemas Linux. +- Se agregó el `clickhouse-test` paquete. Se puede usar para ejecutar pruebas funcionales. +- El archivo tarball de origen ahora se puede publicar en el repositorio. Se puede usar para reproducir la compilación sin usar GitHub. +- Se agregó una integración limitada con Travis CI. Debido a los límites en el tiempo de compilación en Travis, solo se prueba la compilación de depuración y se ejecuta un subconjunto limitado de pruebas. +- Añadido soporte para `Cap'n'Proto` en la compilación predeterminada. +- Se ha cambiado el formato de las fuentes de documentación de `Restricted Text` a `Markdown`. +- Añadido soporte para `systemd` (Vladimir Smirnov). Está deshabilitado por defecto debido a la incompatibilidad con algunas imágenes del sistema operativo y se puede habilitar manualmente. +- Para la generación de código dinámico, `clang` y `lld` están incrustados en el `clickhouse` binario. También se pueden invocar como `clickhouse clang` y `clickhouse lld` . +- Se eliminó el uso de extensiones GNU del código. Habilitado el `-Wextra` opcion. Al construir con `clang` el valor predeterminado es `libc++` en lugar de `libstdc++`. +- Extraer `clickhouse_parsers` y `clickhouse_common_io` bibliotecas para acelerar las compilaciones de varias herramientas. + +#### Cambios incompatibles hacia atrás: {#backward-incompatible-changes-11} + +- El formato de las marcas en `Log` tablas de tipos que contienen `Nullable` columnas se cambió de una manera incompatible con versiones anteriores. Si tiene estas tablas, debe convertirlas a la `TinyLog` escriba antes de iniciar la nueva versión del servidor. Para hacer esto, reemplace `ENGINE = Log` con `ENGINE = TinyLog` en el correspondiente `.sql` archivo en el `metadata` directorio. Si su tabla no tiene `Nullable` o si el tipo de su tabla no es `Log`, entonces usted no tiene que hacer nada. +- Eliminado el `experimental_allow_extended_storage_definition_syntax` configuración. Ahora esta característica está habilitada de forma predeterminada. +- El `runningIncome` función fue renombrada a `runningDifferenceStartingWithFirstvalue` para evitar confusiones. +- Eliminado el `FROM ARRAY JOIN arr` sintaxis cuando ARRAY JOIN se especifica directamente después de FROM sin tabla (Amos Bird). +- Eliminado el `BlockTabSeparated` formato que se utilizó únicamente con fines de demostración. +- Se ha cambiado el formato de estado para las funciones agregadas `varSamp`, `varPop`, `stddevSamp`, `stddevPop`, `covarSamp`, `covarPop`, `corr`. Si ha almacenado estados de estas funciones agregadas en tablas (utilizando `AggregateFunction` tipo de datos o vistas materializadas con los estados correspondientes), por favor escriba a clickhouse-feedback@yandex-team.com. +- En versiones anteriores del servidor había una característica no documentada: si una función agregada depende de parámetros, aún puede especificarla sin parámetros en el tipo de datos AggregateFunction . Ejemplo: `AggregateFunction(quantiles, UInt64)` en lugar de `AggregateFunction(quantiles(0.5, 0.9), UInt64)`. Esta característica se perdió. Aunque no estaba documentado, planeamos apoyarlo nuevamente en futuras versiones. +- Los tipos de datos de enumeración no se pueden usar en funciones de agregado mínimo / máximo. Esta habilidad se devolverá en la próxima versión. + +#### Tenga en cuenta al actualizar: {#please-note-when-upgrading} + +- Al realizar una actualización continua en un clúster, en el momento en que algunas de las réplicas ejecutan la versión anterior de ClickHouse y otras ejecutan la nueva versión, la replicación se detiene temporalmente y el mensaje `unknown parameter 'shard'` aparece en el registro. La replicación continuará después de que se actualicen todas las réplicas del clúster. +- Si se están ejecutando diferentes versiones de ClickHouse en los servidores de clúster, es posible que las consultas distribuidas que utilizan las siguientes funciones tengan resultados incorrectos: `varSamp`, `varPop`, `stddevSamp`, `stddevPop`, `covarSamp`, `covarPop`, `corr`. Debe actualizar todos los nodos del clúster. diff --git a/docs/es/changelog/2019.md b/docs/es/changelog/2019.md deleted file mode 120000 index 105ca144fca..00000000000 --- a/docs/es/changelog/2019.md +++ /dev/null @@ -1 +0,0 @@ -../../en/changelog/2019.md \ No newline at end of file diff --git a/docs/es/changelog/2019.md b/docs/es/changelog/2019.md new file mode 100644 index 00000000000..3acb9db1997 --- /dev/null +++ b/docs/es/changelog/2019.md @@ -0,0 +1,2065 @@ +--- +machine_translated: true +--- + +## ClickHouse de liberación de la v19.17 {#clickhouse-release-v19-17} + +### Lanzamiento de ClickHouse v19.17.6.36, 2019-12-27 {#clickhouse-release-v19-17-6-36-2019-12-27} + +#### Corrección de errores {#bug-fix} + +- Desbordamiento de búfer potencial fijo en descomprimir. El usuario malintencionado puede pasar datos comprimidos fabricados que podrían causar lectura después del búfer. Este problema fue encontrado por Eldar Zaitov del equipo de seguridad de la información de Yandex. [\#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió un posible bloqueo del servidor (`std::terminate`) cuando el servidor no puede enviar o escribir datos en formato JSON o XML con valores de tipo de datos String (que requieren validación UTF-8) o al comprimir datos de resultados con el algoritmo Brotli o en algunos otros casos raros. [\#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Diccionarios fijos con fuente de un clickhouse `VIEW`, ahora leer tales diccionarios no causa el error `There is no query`. [\#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se corrigió la comprobación de si un host cliente está permitido por host\_regexp especificado en los usuarios.XML. [\#8241](https://github.com/ClickHouse/ClickHouse/pull/8241), [\#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) +- `RENAME TABLE` para una tabla distribuida ahora cambia el nombre de la carpeta que contiene los datos insertados antes de enviarlos a los fragmentos. Esto soluciona un problema con los cambios de nombre sucesivos `tableA->tableB`, `tableC->tableA`. [\#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([Tavplubix](https://github.com/tavplubix)) +- `range_hashed` Los diccionarios externos creados por consultas DDL ahora permiten rangos de tipos numéricos arbitrarios. [\#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) +- Fijo `INSERT INTO table SELECT ... FROM mysql(...)` función de la tabla. [\#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([Tavplubix](https://github.com/tavplubix)) +- Segfault fijo en `INSERT INTO TABLE FUNCTION file()` mientras se inserta en un archivo que no existe. Ahora, en este caso, se crearía un archivo y luego se procesaría la inserción. [\#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) +- Se corrigió el error bitmapAnd al intersecar un mapa de bits agregado y un mapa de bits escalar. [\#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) +- Segfault fijo cuando `EXISTS` consulta se utilizó sin `TABLE` o `DICTIONARY` calificador, al igual que `EXISTS t`. [\#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Tipo de retorno fijo para funciones `rand` y `randConstant` en caso de argumento anulable. Ahora las funciones siempre regresan `UInt32` y nunca `Nullable(UInt32)`. [\#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fijo `DROP DICTIONARY IF EXISTS db.dict`, ahora no lanza la excepción si `db` no existe. [\#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) +- Si una tabla no se eliminó por completo debido a un bloqueo del servidor, el servidor intentará restaurarla y cargarla [\#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([Tavplubix](https://github.com/tavplubix)) +- Se corrigió una consulta de recuento trivial para una tabla distribuida si hay más de dos tablas locales de fragmentos. [\#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) +- Se corrigió un error que conducía a una carrera de datos en DB :: BlockStreamProfileInfo :: calculateRowsBeforeLimit() [\#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alejandro Kazakov](https://github.com/Akazz)) +- Fijo `ALTER table MOVE part` se ejecuta inmediatamente después de fusionar la parte especificada, lo que podría provocar el movimiento de una parte en la que la parte especificada se fusionó. Ahora mueve correctamente la parte especificada. [\#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Las expresiones para los diccionarios se pueden especificar como cadenas ahora. Esto es útil para el cálculo de atributos al extraer datos de fuentes que no son de ClickHouse porque permite usar la sintaxis que no son de ClickHouse para esas expresiones. [\#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) +- Se corrigió una carrera muy rara en `clickhouse-copier` debido a un desbordamiento en ZXid. [\#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Más información](https://github.com/dingxiangfei2009)) +- Se corrigió el error cuando después de la consulta falló (debido a “Too many simultaneous queries” por ejemplo) no leería información de tablas externas, y el + La siguiente solicitud interpretaría esta información como el comienzo de la siguiente consulta causando un error como `Unknown packet from client`. [\#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) +- Evite la desreferencia nula después de “Unknown packet X from server” [\#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) +- Restaure el soporte de todas las configuraciones regionales de la UCI, agregue la capacidad de aplicar intercalaciones para expresiones constantes y agregue el nombre del idioma al sistema.mesa de colaciones. [\#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) +- Número de secuencias para leer desde `StorageFile` y `StorageHDFS` ahora está limitado, para evitar exceder el límite de memoria. [\#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) +- Fijo `CHECK TABLE` consulta para `*MergeTree` mesas sin llave. [\#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) +- Eliminado el número de mutación de un nombre de pieza en caso de que no hubiera mutaciones. Esta eliminación mejoró la compatibilidad con versiones anteriores. [\#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) +- Se corrigió el error de que las mutaciones se omiten para algunas partes adjuntas debido a que su versión de datos son más grandes que la versión de mutación de la tabla. [\#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) +- Permita iniciar el servidor con copias redundantes de piezas después de moverlas a otro dispositivo. [\#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corregido el error “Sizes of columns doesn’t match” que pueden aparecer al usar columnas de función agregadas. [\#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) +- Ahora se lanzará una excepción en caso de usar WITH TIES junto con LIMIT BY. Y ahora es posible usar TOP con LIMIT BY. [\#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Repara la recarga del diccionario si tiene `invalidate_query`, que detuvo las actualizaciones y alguna excepción en los intentos de actualización anteriores. [\#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) + +### Lanzamiento de ClickHouse v19.17.4.11, 2019-11-22 {#clickhouse-release-v19-17-4-11-2019-11-22} + +#### Cambio incompatible hacia atrás {#backward-incompatible-change} + +- Usar column en lugar de AST para almacenar resultados de subconsultas escalares para un mejor rendimiento. Configuración `enable_scalar_subquery_optimization` se agregó en 19.17 y se habilitó de forma predeterminada. Conduce a errores como [este](https://github.com/ClickHouse/ClickHouse/issues/7851) durante la actualización a 19.17.2 o 19.17.3 de versiones anteriores. Esta configuración estaba deshabilitada de forma predeterminada en 19.17.4, para hacer posible la actualización desde 19.16 y versiones anteriores sin errores. [\#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos pájaro](https://github.com/amosbird)) + +#### Novedad {#new-feature} + +- Agregue la capacidad de crear diccionarios con consultas DDL. [\#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin)) +- Hacer `bloom_filter` tipo de índice de apoyo `LowCardinality` y `Nullable` [\#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [\#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Añadir función `isValidJSON` para verificar que la cadena pasada sea un json válido. [\#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [\#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir)) +- Implementar `arrayCompact` función [\#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr)) +- Función creada `hex` para números decimales. Funciona como `hex(reinterpretAsString())`, pero no elimina los últimos cero bytes. [\#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb)) +- Añadir `arrayFill` y `arrayReverseFill` funciones, que reemplazan elementos por otros elementos delante / detrás de ellos en la matriz. [\#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([Hcz](https://github.com/hczhcz)) +- Añadir `CRC32IEEE()`/`CRC64()` apoyo [\#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat)) +- Implementar `char` función similar a uno en [mysql](https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_char) [\#7486](https://github.com/ClickHouse/ClickHouse/pull/7486) ([Sundyli](https://github.com/sundy-li)) +- Añadir `bitmapTransform` función. Transforma una matriz de valores en un mapa de bits a otra matriz de valores, el resultado es un nuevo mapa de bits [\#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang)) +- Aplicado `javaHashUTF16LE()` función [\#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([chimbab](https://github.com/achimbab)) +- Añadir `_shard_num` columna virtual para el motor distribuido [\#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat)) + +#### Característica Experimental {#experimental-feature} + +- Soporte para procesadores (nueva canalización de ejecución de consultas) en `MergeTree`. [\#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Corrección de errores {#bug-fix-1} + +- Corregir el análisis flotante incorrecto en `Values` [\#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [\#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([Tavplubix](https://github.com/tavplubix)) +- Soluciona un punto muerto raro que puede ocurrir cuando trace\_log está habilitado. [\#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) +- Evitar la duplicación de mensajes al producir la tabla Kafka tiene cualquier MV seleccionando de ella [\#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) +- Soporte para `Array(LowCardinality(Nullable(String)))` en `IN`. Resolver [\#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [\#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([chimbab](https://github.com/achimbab)) +- Añadir manejo de `SQL_TINYINT` y `SQL_BIGINT`, y fijar el manejo de `SQL_FLOAT` tipos de origen de datos en ODBC Bridge. [\#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) +- Corregir la agregación (`avg` y quantiles) sobre columnas decimales vacías [\#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90)) +- Fijar `INSERT` en Distribuido con `MATERIALIZED` columna [\#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) +- Hacer `MOVE PARTITION` funciona si algunas partes de la partición ya están en el disco o volumen de destino [\#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se corrigió un error con los enlaces duros que no se creaban durante las mutaciones en `ReplicatedMergeTree` en configuraciones de varios discos. [\#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se corrigió un error con una mutación en un MergeTree cuando toda la parte permanece sin cambios y el mejor espacio se encuentra en otro disco [\#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corregido error con `keep_free_space_ratio` no se lee desde la configuración de discos [\#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corregir error con la tabla solo contiene `Tuple` columnas o columnas con rutas complejas. Fijar [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [\#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin)) +- No tener en cuenta la memoria para el motor de búfer en el límite de max\_memory\_usage [\#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat)) +- Corregir el uso de marcas final en `MergeTree` ordenadas por `tuple()`. En casos raros podría conducir a `Can't adjust last granule` error mientras selecciona. [\#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir errores en mutaciones que tienen predicados con acciones que requieren contexto (por ejemplo, funciones para json), lo que puede provocar bloqueos o excepciones extrañas. [\#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin)) +- Corregir la falta de coincidencia de los nombres de bases de datos y tablas que se escapan en `data/` y `shadow/` Directory [\#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alejandro Burmak](https://github.com/Alex-Burmak)) +- Support duplicated keys in RIGHT\|FULL JOINs, e.g. `ON t.x = u.x AND t.x = u.y`. Corregir el bloqueo en este caso. [\#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar `Not found column in block` al unirse en la expresión con RIGHT o FULL JOIN. [\#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2)) +- Un intento más de arreglar bucle infinito en `PrettySpace` formato [\#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia)) +- Corregir error en `concat` función cuando todos los argumentos fueron `FixedString` del mismo tamaño. [\#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin)) +- Se corrigió la excepción en caso de usar 1 argumento al definir almacenes S3, URL y HDFS. [\#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corregir el alcance de InterpreterSelectQuery para vistas con consulta [\#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat)) + +#### Mejora {#improvement} + +- `Nullable` columnas reconocidas y valores NULL manejados correctamente por ODBC-bridge [\#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) +- Escribir el lote actual para el envío distribuido atómicamente [\#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat)) +- Lance una excepción si no podemos detectar la tabla para el nombre de la columna en la consulta. [\#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `merge_max_block_size` ajuste a `MergeTreeSettings` [\#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2)) +- Consultas con `HAVING` y sin `GROUP BY` asumir grupo por constante. Tan, `SELECT 1 HAVING 1` ahora devuelve un resultado. [\#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos pájaro](https://github.com/amosbird)) +- Soporte de análisis `(X,)` como tupla similar a python. [\#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [\#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos pájaro](https://github.com/amosbird)) +- Hacer `range` comportamientos de función casi como uno pitónico. [\#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([Sundyli](https://github.com/sundy-li)) +- Añadir `constraints` columnas a la mesa `system.settings` [\#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar)) +- Mejor formato nulo para el controlador tcp, por lo que es posible usar `select ignore() from table format Null` para la medida de perf a través de clickhouse-client [\#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos pájaro](https://github.com/amosbird)) +- Consultas como `CREATE TABLE ... AS (SELECT (1, 2))` se analizan correctamente [\#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([Hcz](https://github.com/hczhcz)) + +#### Mejora del rendimiento {#performance-improvement} + +- Se mejora el rendimiento de la agregación sobre claves de cadena cortas. [\#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alejandro Kuzmenkov](https://github.com/akuzm), [Amos pájaro](https://github.com/amosbird)) +- Ejecute otra pasada de análisis de sintaxis / expresión para obtener optimizaciones potenciales después de que se plieguen los predicados constantes. [\#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos pájaro](https://github.com/amosbird)) +- Use metainformación de almacenamiento para evaluar trivial `SELECT count() FROM table;` [\#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos pájaro](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov)) +- Vectorizar el procesamiento `arrayReduce` similar a Aggregator `addBatch`. [\#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos pájaro](https://github.com/amosbird)) +- Mejoras menores en el rendimiento de `Kafka` consumo [\#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement} + +- Agregue soporte para la compilación cruzada a la arquitectura de CPU AARCH64. Refactorizar la secuencia de comandos del empaquetador. [\#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [\#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7)) +- Desempaquete las cadenas de herramientas darwin-x86\_64 y linux-aarch64 en el volumen Docker montado al crear paquetes [\#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7)) +- Actualizar la imagen de Docker para Binary Packager [\#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7)) +- Se corrigieron errores de compilación en MacOS Catalina [\#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp)) +- Algunas refactorizaciones en la lógica de análisis de consultas: dividir la clase compleja en varias simples. [\#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2)) +- Reparar la compilación sin submódulos [\#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([propulsor](https://github.com/proller)) +- Mejor `add_globs` en archivos CMake [\#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos pájaro](https://github.com/amosbird)) +- Eliminar rutas codificadas en `unwind` objetivo [\#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok)) +- Permitir usar el formato mysql sin ssl [\#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([propulsor](https://github.com/proller)) + +#### Otro {#other} + +- Añadido gramática ANTLR4 para ClickHouse SQL dialecto [\#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [\#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +## ClickHouse de liberación de la v19.16 {#clickhouse-release-v19-16} + +#### Todos los derechos reservados. {#clickhouse-release-v19-16-14-65-2020-03-05} + +- Corregir la incompatibilidad de subconsultas distribuidas con versiones anteriores de CH. Fijar [\#7851](https://github.com/ClickHouse/ClickHouse/issues/7851) + [(tabplubix)](https://github.com/tavplubix) +- Al ejecutar `CREATE` consulta, doblar expresiones constantes en argumentos del motor de almacenamiento. Reemplace el nombre de la base de datos vacía con la base de datos actual. Fijar [\#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [\#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). También corrige la verificación de la dirección local en `ClickHouseDictionarySource`. + [\#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) +- Ahora el fondo se fusiona en `*MergeTree` familia de motores de mesa preservar el orden de volumen de políticas de almacenamiento con mayor precisión. + [\#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Evitar la pérdida de datos en `Kafka` en casos raros cuando la excepción ocurre después de leer el sufijo pero antes de confirmar. Fijar [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Relacionados: [\#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) + [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) +- Corregir un error que conduce a la terminación del servidor al intentar usar / soltar `Kafka` tabla creada con parámetros incorrectos. Fijar [\#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporar [\#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). + [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) +- Permitir usar `MaterializedView` con subconsultas anteriores `Kafka` tabla. + [\#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) + +#### Novedad {#new-feature-1} + +- Añadir `deduplicate_blocks_in_dependent_materialized_views` opción para controlar el comportamiento de las inserciones idempotentes en tablas con vistas materializadas. Esta nueva característica se agregó a la versión de corrección de errores mediante una solicitud especial de Altinity. + [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) + +### Lanzamiento de ClickHouse v19.16.2.2, 2019-10-30 {#clickhouse-release-v19-16-2-2-2019-10-30} + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-1} + +- Agregue la validación de arity faltante para count/counIf . + [\#7095](https://github.com/ClickHouse/ClickHouse/issues/7095) + [\#7298](https://github.com/ClickHouse/ClickHouse/pull/7298) ([Vdimir](https://github.com/Vdimir)) +- Eliminar heredado `asterisk_left_columns_only` configuración (que estaba deshabilitado por defecto). + [\#7335](https://github.com/ClickHouse/ClickHouse/pull/7335) ([Artem + Zuikov](https://github.com/4ertus2)) +- Las cadenas de formato para el formato de datos de la plantilla ahora se especifican en los archivos. + [\#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) + ([Tavplubix](https://github.com/tavplubix)) + +#### Novedad {#new-feature-2} + +- Introduzca uniqCombined64() para calcular la cardinalidad mayor que UINT\_MAX. + [\#7213](https://github.com/ClickHouse/ClickHouse/pull/7213), + [\#7222](https://github.com/ClickHouse/ClickHouse/pull/7222) ([Azat + Khuzhin](https://github.com/azat)) +- Admite índices de filtro Bloom en columnas de matriz. + [\#6984](https://github.com/ClickHouse/ClickHouse/pull/6984) + ([chimbab](https://github.com/achimbab)) +- Agregar una función `getMacro(name)` que devuelve String con el valor de `` + desde la configuración del servidor. [\#7240](https://github.com/ClickHouse/ClickHouse/pull/7240) + ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Establezca dos opciones de configuración para un diccionario basado en un origen HTTP: `credentials` y + `http-headers`. [\#7092](https://github.com/ClickHouse/ClickHouse/pull/7092) ([Guillaume + Tassery](https://github.com/YiuRULE)) +- Agregar un nuevo ProfileEvent `Merge` que cuenta el número de fusiones de fondo lanzadas. + [\#7093](https://github.com/ClickHouse/ClickHouse/pull/7093) ([Mijaíl + Korotov](https://github.com/millb)) +- Agregue la función fullHostName que devuelve un nombre de dominio completo. + [\#7263](https://github.com/ClickHouse/ClickHouse/issues/7263) + [\#7291](https://github.com/ClickHouse/ClickHouse/pull/7291) ([Sundyli](https://github.com/sundy-li)) +- Añadir función `arraySplit` y `arrayReverseSplit` que dividió una matriz por “cut off” + condición. Son útiles en el manejo de secuencias de tiempo. + [\#7294](https://github.com/ClickHouse/ClickHouse/pull/7294) ([Hcz](https://github.com/hczhcz)) +- Agregue nuevas funciones que devuelvan la matriz de todos los índices coincidentes en la familia de funciones multiMatch. + [\#7299](https://github.com/ClickHouse/ClickHouse/pull/7299) ([Danila + Kutenin](https://github.com/danlark1)) +- Agregar un nuevo motor de base de datos `Lazy` que está optimizado para almacenar un gran número de pequeños -Log + tabla. [\#7171](https://github.com/ClickHouse/ClickHouse/pull/7171) ([Nosotros + Vasilev](https://github.com/nikvas0)) +- Agregue funciones agregadas groupBitmapAnd, -Or, -Xor para columnas de mapa de bits. [\#7109](https://github.com/ClickHouse/ClickHouse/pull/7109) ([Zhichang + Yu](https://github.com/yuzhichang)) +- Agregue los combinadores de funciones agregadas -OrNull y -OrDefault , que devuelven null + o valores predeterminados cuando no hay nada que agregar. + [\#7331](https://github.com/ClickHouse/ClickHouse/pull/7331) + ([Hcz](https://github.com/hczhcz)) +- Introduzca el formato de datos CustomSeparated que admite el escape personalizado y + reglas delimitador. [\#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) + ([Tavplubix](https://github.com/tavplubix)) +- Soporte Redis como fuente de diccionario externo. [\#4361](https://github.com/ClickHouse/ClickHouse/pull/4361) [\#6962](https://github.com/ClickHouse/ClickHouse/pull/6962) ([comunodi](https://github.com/comunodi), [Anton + Popov](https://github.com/CurtizJ)) + +#### Corrección de errores {#bug-fix-2} + +- Repara el resultado de la consulta incorrecta si tiene `WHERE IN (SELECT ...)` sección y `optimize_read_in_order` ser + utilizar. [\#7371](https://github.com/ClickHouse/ClickHouse/pull/7371) ([Anton + Popov](https://github.com/CurtizJ)) +- Complemento de autenticación MariaDB deshabilitado, que depende de archivos fuera del proyecto. + [\#7140](https://github.com/ClickHouse/ClickHouse/pull/7140) ([Yuriy + Baranov](https://github.com/yurriy)) +- Solucionar excepción `Cannot convert column ... because it is constant but values of constants are different in source and result` que rara vez podría suceder cuando funciones `now()`, `today()`, + `yesterday()`, `randConstant()` se utilizan. + [\#7156](https://github.com/ClickHouse/ClickHouse/pull/7156) ([Nikolai + Kochetov](https://github.com/KochetovNicolai)) +- Solucionado el problema de usar HTTP keep alive timeout en lugar de TCP keep alive timeout. + [\#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily + Nemkov](https://github.com/Enmk)) +- Se corrigió un error de segmentación en groupBitmapOr (problema [\#7109](https://github.com/ClickHouse/ClickHouse/issues/7109)). + [\#7289](https://github.com/ClickHouse/ClickHouse/pull/7289) ([Zhichang + Yu](https://github.com/yuzhichang)) +- Para las vistas materializadas, se llama a la confirmación para Kafka después de que se escribieron todos los datos. + [\#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) +- Corregido mal `duration_ms` valor en `system.part_log` tabla. Fue diez veces fuera. + [\#7172](https://github.com/ClickHouse/ClickHouse/pull/7172) ([Vladimir + Chebotarev](https://github.com/excitoon)) +- Una solución rápida para resolver el bloqueo en la tabla LIVE VIEW y volver a habilitar todas las pruebas LIVE VIEW. + [\#7201](https://github.com/ClickHouse/ClickHouse/pull/7201) + ([vzakaznikov](https://github.com/vzakaznikov)) +- Serialice los valores NULL correctamente en los índices mínimos / máximos de las partes MergeTree. + [\#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alejandro + Kuzmenkov](https://github.com/akuzm)) +- No coloque columnas virtuales en .sql cuando la tabla se crea como `CREATE TABLE AS`. + [\#7183](https://github.com/ClickHouse/ClickHouse/pull/7183) ([Ivan](https://github.com/abyss7)) +- Fijar falla de segmentación en `ATTACH PART` consulta. + [\#7185](https://github.com/ClickHouse/ClickHouse/pull/7185) + ([alesapin](https://github.com/alesapin)) +- Corrija el resultado incorrecto para algunas consultas dadas por la optimización de las subconsultas IN vacías y vacías + INNER/RIGHT JOIN. [\#7284](https://github.com/ClickHouse/ClickHouse/pull/7284) ([Nikolai + Kochetov](https://github.com/KochetovNicolai)) +- Reparación del error AddressSanitizer en el método LIVE VIEW getHeader (). + [\#7271](https://github.com/ClickHouse/ClickHouse/pull/7271) + ([vzakaznikov](https://github.com/vzakaznikov)) + +#### Mejora {#improvement-1} + +- Añadir un mensaje en caso de queue\_wait\_max\_ms espera se lleva a cabo. + [\#7390](https://github.com/ClickHouse/ClickHouse/pull/7390) ([Azat + Khuzhin](https://github.com/azat)) +- Ajuste hecho `s3_min_upload_part_size` a nivel de mesa. + [\#7059](https://github.com/ClickHouse/ClickHouse/pull/7059) ([Vladimir + Chebotarev](https://github.com/excitoon)) +- Compruebe TTL en StorageFactory. [\#7304](https://github.com/ClickHouse/ClickHouse/pull/7304) + ([Sundyli](https://github.com/sundy-li)) +- Squash bloques de la izquierda en combinación de fusión parcial (optimización). + [\#7122](https://github.com/ClickHouse/ClickHouse/pull/7122) ([Artem + Zuikov](https://github.com/4ertus2)) +- No permita funciones no deterministas en mutaciones de motores de tabla replicados, porque esto + puede introducir inconsistencias entre réplicas. + [\#7247](https://github.com/ClickHouse/ClickHouse/pull/7247) ([Alejandro + Kazakov](https://github.com/Akazz)) +- Deshabilite el rastreador de memoria mientras convierte el seguimiento de la pila de excepciones en cadena. Puede prevenir la pérdida + de mensajes de error de tipo `Memory limit exceeded` en el servidor, lo que `Attempt to read after eof` excepción en el cliente. [\#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) + ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Mejoras de formato varios. Resolver + [\#6033](https://github.com/ClickHouse/ClickHouse/issues/6033), + [\#2633](https://github.com/ClickHouse/ClickHouse/issues/2633), + [\#6611](https://github.com/ClickHouse/ClickHouse/issues/6611), + [\#6742](https://github.com/ClickHouse/ClickHouse/issues/6742) + [\#7215](https://github.com/ClickHouse/ClickHouse/pull/7215) + ([Tavplubix](https://github.com/tavplubix)) +- ClickHouse ignora los valores en el lado derecho del operador IN que no son convertibles a la izquierda + side type. Make it work properly for compound types – Array and Tuple. + [\#7283](https://github.com/ClickHouse/ClickHouse/pull/7283) ([Alejandro + Kuzmenkov](https://github.com/akuzm)) +- Apoyar las desigualdades que faltan para ASOF JOIN. Es posible unir una variante menor o igual y estricta + mayores y menos variantes para la columna ASOF en la sintaxis ON. + [\#7282](https://github.com/ClickHouse/ClickHouse/pull/7282) ([Artem + Zuikov](https://github.com/4ertus2)) +- Optimizar la combinación de fusión parcial. [\#7070](https://github.com/ClickHouse/ClickHouse/pull/7070) + ([Artem Zuikov](https://github.com/4ertus2)) +- No utilice más de 98K de memoria en funciones uniqCombined. + [\#7236](https://github.com/ClickHouse/ClickHouse/pull/7236), + [\#7270](https://github.com/ClickHouse/ClickHouse/pull/7270) ([Azat + Khuzhin](https://github.com/azat)) +- Enjuague las partes de la tabla de unión de la derecha en el disco en PartialMergeJoin (si no hay suficiente + memoria). Cargue los datos cuando sea necesario. [\#7186](https://github.com/ClickHouse/ClickHouse/pull/7186) + ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejora del rendimiento {#performance-improvement-1} + +- Acelere joinGet con argumentos const evitando la duplicación de datos. + [\#7359](https://github.com/ClickHouse/ClickHouse/pull/7359) ([Amos + Ave](https://github.com/amosbird)) +- Regrese temprano si la subconsulta está vacía. + [\#7007](https://github.com/ClickHouse/ClickHouse/pull/7007) ([小路](https://github.com/nicelulu)) +- Optimizar el análisis de la expresión SQL en Valores. + [\#6781](https://github.com/ClickHouse/ClickHouse/pull/6781) + ([Tavplubix](https://github.com/tavplubix)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-1} + +- Deshabilite algunas contribuciones para la compilación cruzada en Mac OS. + [\#7101](https://github.com/ClickHouse/ClickHouse/pull/7101) ([Ivan](https://github.com/abyss7)) +- Agregue enlaces faltantes con PocoXML para clickhouse\_common\_io. + [\#7200](https://github.com/ClickHouse/ClickHouse/pull/7200) ([Azat + Khuzhin](https://github.com/azat)) +- Acepte varios argumentos de filtro de prueba en clickhouse-test. + [\#7226](https://github.com/ClickHouse/ClickHouse/pull/7226) ([Alejandro + Kuzmenkov](https://github.com/akuzm)) +- Habilitar musl y jemalloc para ARM. [\#7300](https://github.com/ClickHouse/ClickHouse/pull/7300) + ([Amos pájaro](https://github.com/amosbird)) +- Añadir `--client-option` parámetro para `clickhouse-test` para pasar parámetros adicionales al cliente. + [\#7277](https://github.com/ClickHouse/ClickHouse/pull/7277) ([Nikolai + Kochetov](https://github.com/KochetovNicolai)) +- Conservar las configuraciones existentes en la actualización del paquete rpm. + [\#7103](https://github.com/ClickHouse/ClickHouse/pull/7103) + ([filimonov](https://github.com/filimonov)) +- Corregir errores detectados por PVS. [\#7153](https://github.com/ClickHouse/ClickHouse/pull/7153) ([Artem + Zuikov](https://github.com/4ertus2)) +- Corregir la compilación para Darwin. [\#7149](https://github.com/ClickHouse/ClickHouse/pull/7149) + ([Ivan](https://github.com/abyss7)) +- glibc 2.29 compatibilidad. [\#7142](https://github.com/ClickHouse/ClickHouse/pull/7142) ([Amos + Ave](https://github.com/amosbird)) +- Asegúrese de que dh\_clean no toque los archivos fuente potenciales. + [\#7205](https://github.com/ClickHouse/ClickHouse/pull/7205) ([Amos + Ave](https://github.com/amosbird)) +- Intente evitar conflictos al actualizar desde rpm de altinidad: tiene un archivo de configuración empaquetado por separado + en clickhouse-server-common. [\#7073](https://github.com/ClickHouse/ClickHouse/pull/7073) + ([filimonov](https://github.com/filimonov)) +- Optimice algunos archivos de encabezado para reconstrucciones más rápidas. + [\#7212](https://github.com/ClickHouse/ClickHouse/pull/7212), + [\#7231](https://github.com/ClickHouse/ClickHouse/pull/7231) ([Alejandro + Kuzmenkov](https://github.com/akuzm)) +- Agregue pruebas de rendimiento para Date y DateTime. [\#7332](https://github.com/ClickHouse/ClickHouse/pull/7332) ([Vasily + Nemkov](https://github.com/Enmk)) +- Corregir algunas pruebas que contenían mutaciones no deterministas. + [\#7132](https://github.com/ClickHouse/ClickHouse/pull/7132) ([Alejandro + Kazakov](https://github.com/Akazz)) +- Agregue compilación con MemorySanitizer a CI. [\#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) + ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Evite el uso de valores no inicializados en MetricsTransmitter. + [\#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat + Khuzhin](https://github.com/azat)) +- Solucionar algunos problemas en los campos encontrados por MemorySanitizer. + [\#7135](https://github.com/ClickHouse/ClickHouse/pull/7135), + [\#7179](https://github.com/ClickHouse/ClickHouse/pull/7179) ([Alejandro + Kuzmenkov](https://github.com/akuzm)), [\#7376](https://github.com/ClickHouse/ClickHouse/pull/7376) + ([Amos pájaro](https://github.com/amosbird)) +- Corrige el comportamiento indefinido en murmurhash32. [\#7388](https://github.com/ClickHouse/ClickHouse/pull/7388) ([Amos + Ave](https://github.com/amosbird)) +- Corrige el comportamiento indefinido en StoragesInfoStream. [\#7384](https://github.com/ClickHouse/ClickHouse/pull/7384) + ([Tavplubix](https://github.com/tavplubix)) +- Se corrigieron expresiones constantes plegables para motores de bases de datos externas (MySQL, ODBC, JDBC). En anteriores + versiones no funcionaba para múltiples expresiones constantes y no funcionaba en absoluto para Date, + DateTime y UUID. Esto corrige [\#7245](https://github.com/ClickHouse/ClickHouse/issues/7245) + [\#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) + ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparación del error de carrera de datos ThreadSanitizer en LIVE VIEW al acceder a la variable no\_users\_thread. + [\#7353](https://github.com/ClickHouse/ClickHouse/pull/7353) + ([vzakaznikov](https://github.com/vzakaznikov)) +- Deshacerse de los símbolos malloc en libcommon + [\#7134](https://github.com/ClickHouse/ClickHouse/pull/7134), + [\#7065](https://github.com/ClickHouse/ClickHouse/pull/7065) ([Amos + Ave](https://github.com/amosbird)) +- Agregue el indicador global ENABLE\_LIBRARIES para deshabilitar todas las bibliotecas. + [\#7063](https://github.com/ClickHouse/ClickHouse/pull/7063) + ([propulsor](https://github.com/proller)) + +#### Limpieza de código {#code-cleanup} + +- Generalice el repositorio de configuración para prepararse para DDL para diccionarios. [\#7155](https://github.com/ClickHouse/ClickHouse/pull/7155) + ([alesapin](https://github.com/alesapin)) +- Analizador de diccionarios DDL sin ninguna semántica. + [\#7209](https://github.com/ClickHouse/ClickHouse/pull/7209) + ([alesapin](https://github.com/alesapin)) +- Divida ParserCreateQuery en diferentes analizadores más pequeños. + [\#7253](https://github.com/ClickHouse/ClickHouse/pull/7253) + ([alesapin](https://github.com/alesapin)) +- Pequeña refactorización y cambio de nombre cerca de diccionarios externos. + [\#7111](https://github.com/ClickHouse/ClickHouse/pull/7111) + ([alesapin](https://github.com/alesapin)) +- Refactorizar el código para prepararse para el papel de control de acceso basado. [\#7235](https://github.com/ClickHouse/ClickHouse/pull/7235) ([Vitaly + Baranov](https://github.com/vitlibar)) +- Algunas mejoras en el código DatabaseOrdinary. + [\#7086](https://github.com/ClickHouse/ClickHouse/pull/7086) ([Nosotros + Vasilev](https://github.com/nikvas0)) +- No use iteradores en los métodos find() y emplace() de tablas hash. + [\#7026](https://github.com/ClickHouse/ClickHouse/pull/7026) ([Alejandro + Kuzmenkov](https://github.com/akuzm)) +- Repare getMultipleValuesFromConfig en caso de que la raíz del parámetro no esté vacía. [\#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) + ([Mikhail Korotov](https://github.com/millb)) +- Eliminar algunos copiar y pegar (TemporaryFile y TemporaryFileStream) + [\#7166](https://github.com/ClickHouse/ClickHouse/pull/7166) ([Artem + Zuikov](https://github.com/4ertus2)) +- Mejora de la legibilidad del código un poco (`MergeTreeData::getActiveContainingPart`). + [\#7361](https://github.com/ClickHouse/ClickHouse/pull/7361) ([Vladimir + Chebotarev](https://github.com/excitoon)) +- Espere a que todos los trabajos programados, que utilizan objetos locales, si `ThreadPool::schedule(...)` lanzar + salvedad. Cambiar nombre `ThreadPool::schedule(...)` a `ThreadPool::scheduleOrThrowOnError(...)` y + corregir los comentarios para hacer obvio que puede lanzar. + [\#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) + ([Tavplubix](https://github.com/tavplubix)) + +## Lanzamiento de ClickHouse 19.15 {#clickhouse-release-19-15} + +### Lanzamiento de ClickHouse 19.15.4.10, 31.10.2019 {#clickhouse-release-19-15-4-10-2019-10-31} + +#### Corrección de errores {#bug-fix-3} + +- Se agregó el manejo de SQL\_TINYINT y SQL\_BIGINT, y el manejo de correcciones de los tipos de origen de datos SQL\_FLOAT en ODBC Bridge. + [\#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) +- Se permite tener algunas partes en el disco de destino o el volumen en MOVE PARTITION. + [\#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se corrigieron valores NULL en columnas anulables a través del puente ODBC. + [\#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) +- Se corrigió INSERT en el nodo distribuido no local con columnas MATERIALIZED. + [\#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat)) +- Función fija getMultipleValuesFromConfig. + [\#7374](https://github.com/ClickHouse/ClickHouse/pull/7374) ([Mikhail Korotov](https://github.com/millb)) +- Solucionado el problema de usar HTTP keep alive timeout en lugar de TCP keep alive timeout. + [\#7351](https://github.com/ClickHouse/ClickHouse/pull/7351) ([Vasily Nemkov](https://github.com/Enmk)) +- Espere a que todos los trabajos finalicen con excepción (corrige fallas raras). + [\#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([Tavplubix](https://github.com/tavplubix)) +- No presione a MVs cuando inserte en la tabla Kafka. + [\#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) +- Deshabilitar el rastreador de memoria para la pila de excepciones. + [\#7264](https://github.com/ClickHouse/ClickHouse/pull/7264) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se corrigió el código incorrecto en la transformación de la consulta para la base de datos externa. + [\#7252](https://github.com/ClickHouse/ClickHouse/pull/7252) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite el uso de valores no inicializados en MetricsTransmitter. + [\#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) +- Se agregó configuración de ejemplo con macros para pruebas ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.15.3.6, 2019-10-09 {#clickhouse-release-19-15-3-6-2019-10-09} + +#### Corrección de errores {#bug-fix-4} + +- Se corrigió bad\_variant en el diccionario hash. + ([alesapin](https://github.com/alesapin)) +- Se corrigió un error con un error de segmentación en la consulta ATTACH PART. + ([alesapin](https://github.com/alesapin)) +- Cálculo de tiempo fijo en `MergeTreeData`. + ([Vladimir Chebotarev](https://github.com/excitoon)) +- Comprometerse con Kafka explícitamente después de finalizar la escritura. + [\#7175](https://github.com/ClickHouse/ClickHouse/pull/7175) ([Ivan](https://github.com/abyss7)) +- Serialice los valores NULL correctamente en los índices mínimos / máximos de las partes MergeTree. + [\#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alejandro Kuzmenkov](https://github.com/akuzm)) + +### Lanzamiento de ClickHouse 19.15.2.2, 2019-10-01 {#clickhouse-release-19-15-2-2-2019-10-01} + +#### Novedad {#new-feature-3} + +- Almacenamiento por niveles: admite el uso de múltiples volúmenes de almacenamiento para tablas con el motor MergeTree. Es posible almacenar datos nuevos en SSD y mover automáticamente datos antiguos a HDD. ([ejemplo](https://clickhouse.github.io/clickhouse-presentations/meetup30/new_features/#12)). [\#4918](https://github.com/ClickHouse/ClickHouse/pull/4918) ([Igr](https://github.com/ObjatieGroba)) [\#6489](https://github.com/ClickHouse/ClickHouse/pull/6489) ([alesapin](https://github.com/alesapin)) +- Agregar función de tabla `input` para leer los datos entrantes en `INSERT SELECT` consulta. [\#5450](https://github.com/ClickHouse/ClickHouse/pull/5450) ([Palasonic1](https://github.com/palasonic1)) [\#6832](https://github.com/ClickHouse/ClickHouse/pull/6832) ([Anton Popov](https://github.com/CurtizJ)) +- Añadir un `sparse_hashed` diccionario, que es funcionalmente equivalente al `hashed` diseño, pero es más eficiente en la memoria. Utiliza aproximadamente el doble de menos memoria a costa de una recuperación de valor más lenta. [\#6894](https://github.com/ClickHouse/ClickHouse/pull/6894) ([Azat Khuzhin](https://github.com/azat)) +- Implementar la capacidad de definir la lista de usuarios para el acceso a los diccionarios. Sólo la base de datos conectada actual utilizando. [\#6907](https://github.com/ClickHouse/ClickHouse/pull/6907) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Añadir `LIMIT` opción para `SHOW` consulta. [\#6944](https://github.com/ClickHouse/ClickHouse/pull/6944) ([Filipina Malkovsky](https://github.com/malkfilipp)) +- Añadir `bitmapSubsetLimit(bitmap, range_start, limit)` función, que devuelve subconjunto de la más pequeña `limit` valores en el conjunto que no es menor que `range_start`. [\#6957](https://github.com/ClickHouse/ClickHouse/pull/6957) ([Zhichang Yu](https://github.com/yuzhichang)) +- Añadir `bitmapMin` y `bitmapMax` función. [\#6970](https://github.com/ClickHouse/ClickHouse/pull/6970) ([Zhichang Yu](https://github.com/yuzhichang)) +- Añadir función `repeat` relacionado con [Información detallada](https://github.com/ClickHouse/ClickHouse/issues/6648) [\#6999](https://github.com/ClickHouse/ClickHouse/pull/6999) ([Más información](https://github.com/ucasFL)) + +#### Característica Experimental {#experimental-feature-1} + +- Implementar (en memoria) Combinar unir variante que no cambia la canalización actual. El resultado está parcialmente ordenado por clave de combinación. Establecer `partial_merge_join = 1` para usar esta función. La combinación de combinación todavía está en desarrollo. [\#6940](https://github.com/ClickHouse/ClickHouse/pull/6940) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `S3` función del motor y de la tabla. Todavía está en desarrollo (todavía no hay soporte de autenticación). [\#5596](https://github.com/ClickHouse/ClickHouse/pull/5596) ([Vladimir Chebotarev](https://github.com/excitoon)) + +#### Mejora {#improvement-2} + +- Cada mensaje leído de Kafka se inserta atómicamente. Esto resuelve casi todos los problemas conocidos con el motor Kafka. [\#6950](https://github.com/ClickHouse/ClickHouse/pull/6950) ([Ivan](https://github.com/abyss7)) +- Mejoras para la conmutación por error de consultas distribuidas. Acortar el tiempo de recuperación, también ahora es configurable y se puede ver en `system.clusters`. [\#6399](https://github.com/ClickHouse/ClickHouse/pull/6399) ([Vasily Nemkov](https://github.com/Enmk)) +- Soporta valores numéricos para enumeraciones directamente en `IN` apartado. \#6766 [\#6941](https://github.com/ClickHouse/ClickHouse/pull/6941) ([Dimarub2000](https://github.com/dimarub2000)) +- El soporte (opcional, deshabilitado de forma predeterminada) redirige el almacenamiento de URL. [\#6914](https://github.com/ClickHouse/ClickHouse/pull/6914) ([Más información](https://github.com/maqroll)) +- Agregue un mensaje de información cuando el cliente con una versión anterior se conecte a un servidor. [\#6893](https://github.com/ClickHouse/ClickHouse/pull/6893) ([Filipina Malkovsky](https://github.com/malkfilipp)) +- Eliminar el límite máximo de tiempo de suspensión de interrupción para el envío de datos en tablas distribuidas [\#6895](https://github.com/ClickHouse/ClickHouse/pull/6895) ([Azat Khuzhin](https://github.com/azat)) +- Agregue la capacidad de enviar eventos de perfil (contadores) con valores acumulativos al grafito. Se puede habilitar bajo `` en el servidor `config.xml`. [\#6969](https://github.com/ClickHouse/ClickHouse/pull/6969) ([Azat Khuzhin](https://github.com/azat)) +- Añadir tipo de fundición automática `T` a `LowCardinality(T)` mientras inserta datos en la columna de tipo `LowCardinality(T)` en formato nativo a través de HTTP. [\#6891](https://github.com/ClickHouse/ClickHouse/pull/6891) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Añadir la capacidad de utilizar la función `hex` sin usar `reinterpretAsString` para `Float32`, `Float64`. [\#7024](https://github.com/ClickHouse/ClickHouse/pull/7024) ([Mikhail Korotov](https://github.com/millb)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-2} + +- Agregue gdb-index al binario de clickhouse con información de depuración. Acelerará el tiempo de inicio de `gdb`. [\#6947](https://github.com/ClickHouse/ClickHouse/pull/6947) ([alesapin](https://github.com/alesapin)) +- Acelerar el embalaje deb con parcheado dpkg-deb que utiliza `pigz`. [\#6960](https://github.com/ClickHouse/ClickHouse/pull/6960) ([alesapin](https://github.com/alesapin)) +- Establecer `enable_fuzzing = 1` para habilitar la instrumentación libfuzzer de todo el código del proyecto. [\#7042](https://github.com/ClickHouse/ClickHouse/pull/7042) ([kyprizel](https://github.com/kyprizel)) +- Añadir prueba de humo de construcción dividida en CI. [\#7061](https://github.com/ClickHouse/ClickHouse/pull/7061) ([alesapin](https://github.com/alesapin)) +- Agregue compilación con MemorySanitizer a CI. [\#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Reemplazar `libsparsehash` con `sparsehash-c11` [\#6965](https://github.com/ClickHouse/ClickHouse/pull/6965) ([Azat Khuzhin](https://github.com/azat)) + +#### Corrección de errores {#bug-fix-5} + +- Se corrigió la degradación del rendimiento del análisis de índices en claves complejas en tablas grandes. Esto corrige \#6924. [\#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir el error lógico que causa segfaults al seleccionar el tema vacío de Kafka. [\#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +- Arreglar la conexión MySQL demasiado pronto cerca `MySQLBlockInputStream.cpp`. [\#6882](https://github.com/ClickHouse/ClickHouse/pull/6882) ([Clément Rodriguez](https://github.com/clemrodriguez)) +- Soporte devuelto para núcleos Linux muy antiguos (solución [\#6841](https://github.com/ClickHouse/ClickHouse/issues/6841)) [\#6853](https://github.com/ClickHouse/ClickHouse/pull/6853) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Solucionar la posible pérdida de datos en `insert select` consulta en caso de bloque vacío en el flujo de entrada. \#6834 \#6862 [\#6911](https://github.com/ClickHouse/ClickHouse/pull/6911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Solución para la función `АrrayEnumerateUniqRanked` con matrices vacías en params [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([propulsor](https://github.com/proller)) +- Solucione consultas complejas con combinaciones de matriz y subconsultas globales. [\#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) +- Fijar `Unknown identifier` error en ORDER BY y GROUP BY con múltiples JOINs [\#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijo `MSan` advertencia al ejecutar la función con `LowCardinality` argumento. [\#7062](https://github.com/ClickHouse/ClickHouse/pull/7062) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-2} + +- Se ha cambiado el formato de serialización de los estados de la función agregada de mapa de bits \* para mejorar el rendimiento. No se pueden leer los estados serializados de bitmap\* de versiones anteriores. [\#6908](https://github.com/ClickHouse/ClickHouse/pull/6908) ([Zhichang Yu](https://github.com/yuzhichang)) + +## Lanzamiento de ClickHouse 19.14 {#clickhouse-release-19-14} + +### Lanzamiento de ClickHouse 19.14.7.15, 2019-10-02 {#clickhouse-release-19-14-7-15-2019-10-02} + +#### Corrección de errores {#bug-fix-6} + +- Esta versión también contiene todas las correcciones de errores de 19.11.12.69. +- Compatibilidad fija para consultas distribuidas entre 19.14 y versiones anteriores. Esto corrige [\#7068](https://github.com/ClickHouse/ClickHouse/issues/7068). [\#7069](https://github.com/ClickHouse/ClickHouse/pull/7069) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.14.6.12, 2019-09-19 {#clickhouse-release-19-14-6-12-2019-09-19} + +#### Corrección de errores {#bug-fix-7} + +- Solución para la función `АrrayEnumerateUniqRanked` con matrices vacías en params. [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([propulsor](https://github.com/proller)) +- Nombre de subconsulta fijo en consultas con `ARRAY JOIN` y `GLOBAL IN subquery` con alias. Utilice el alias de subconsulta para el nombre de tabla externo si se especifica. [\#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-3} + +- Fijar [aleteo](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) prueba `00715_fetch_merged_or_mutated_part_zookeeper` reescribiéndolo en scripts de shell porque necesita esperar a que se apliquen las mutaciones. [\#6977](https://github.com/ClickHouse/ClickHouse/pull/6977) ([Alejandro Kazakov](https://github.com/Akazz)) +- Fijo UBSan y MemSan fallo en la función `groupUniqArray` con el argumento de matriz emtpy. Fue causado por la colocación de vacío `PaddedPODArray` en la celda cero de la tabla hash porque no se llamó al constructor para el valor de la celda cero. [\#6937](https://github.com/ClickHouse/ClickHouse/pull/6937) ([Amos pájaro](https://github.com/amosbird)) + +### Lanzamiento de ClickHouse 19.14.3.3, 2019-09-10 {#clickhouse-release-19-14-3-3-2019-09-10} + +#### Novedad {#new-feature-4} + +- `WITH FILL` modificador para `ORDER BY`. (continuación de [\#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [\#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) +- `WITH TIES` modificador para `LIMIT`. (continuación de [\#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [\#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) +- Analizar unquoted `NULL` literal como NULL (si establece `format_csv_unquoted_null_literal_as_null=1`). Inicialice los campos nulos con valores predeterminados si el tipo de datos de este campo no se puede anular (si `input_format_null_as_default=1`). [\#5990](https://github.com/ClickHouse/ClickHouse/issues/5990) [\#6055](https://github.com/ClickHouse/ClickHouse/pull/6055) ([Tavplubix](https://github.com/tavplubix)) +- Soporte para comodines en rutas de funciones de tabla `file` y `hdfs`. Si la ruta contiene comodines, la tabla será de solo lectura. Ejemplo de uso: `select * from hdfs('hdfs://hdfs1:9000/some_dir/another_dir/*/file{0..9}{0..9}')` y `select * from file('some_dir/{some_file,another_file,yet_another}.tsv', 'TSV', 'value UInt32')`. [\#6092](https://github.com/ClickHouse/ClickHouse/pull/6092) ([Olga Khvostikova](https://github.com/stavrolia)) +- Nuevo `system.metric_log` tabla que almacena los valores de `system.events` y `system.metrics` con el intervalo de tiempo especificado. [\#6363](https://github.com/ClickHouse/ClickHouse/issues/6363) [\#6467](https://github.com/ClickHouse/ClickHouse/pull/6467) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [\#6530](https://github.com/ClickHouse/ClickHouse/pull/6530) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir escribir registros de texto de ClickHouse en `system.text_log` tabla. [\#6037](https://github.com/ClickHouse/ClickHouse/issues/6037) [\#6103](https://github.com/ClickHouse/ClickHouse/pull/6103) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [\#6164](https://github.com/ClickHouse/ClickHouse/pull/6164) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mostrar símbolos privados en trazas de pila (esto se hace mediante el análisis de tablas de símbolos de archivos ELF). Se agregó información sobre el archivo y el número de línea en los seguimientos de la pila si la información de depuración está presente. Búsqueda de nombre de símbolo de aceleración con símbolos de indexación presentes en el programa. Se agregaron nuevas funciones SQL para la introspección: `demangle` y `addressToLine`. Función renombrada `symbolizeAddress` a `addressToSymbol` para la consistencia. Función `addressToSymbol` devolverá el nombre destrozado por razones de rendimiento y debe aplicar `demangle`. Añadido `allow_introspection_functions` que está desactivado por defecto. [\#6201](https://github.com/ClickHouse/ClickHouse/pull/6201) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Función de la tabla `values` (el nombre no distingue entre mayúsculas y minúsculas). Permite leer de `VALUES` lista propuesta en [\#5984](https://github.com/ClickHouse/ClickHouse/issues/5984). Ejemplo: `SELECT * FROM VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three'))`. [\#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [\#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([Dimarub2000](https://github.com/dimarub2000)) +- Se agregó la capacidad de alterar la configuración de almacenamiento. Sintaxis: `ALTER TABLE
MODIFY SETTING = `. [\#6366](https://github.com/ClickHouse/ClickHouse/pull/6366) [\#6669](https://github.com/ClickHouse/ClickHouse/pull/6669) [\#6685](https://github.com/ClickHouse/ClickHouse/pull/6685) ([alesapin](https://github.com/alesapin)) +- Soporte para la eliminación de piezas separadas. Sintaxis: `ALTER TABLE DROP DETACHED PART ''`. [\#6158](https://github.com/ClickHouse/ClickHouse/pull/6158) ([Tavplubix](https://github.com/tavplubix)) +- Restricciones de tabla. Permite agregar restricciones a la definición de la tabla que se verificará en la inserción. [\#5273](https://github.com/ClickHouse/ClickHouse/pull/5273) ([Gleb Novikov](https://github.com/NanoBjorn)) [\#6652](https://github.com/ClickHouse/ClickHouse/pull/6652) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte para vistas materializadas en cascada. [\#6324](https://github.com/ClickHouse/ClickHouse/pull/6324) ([Amos pájaro](https://github.com/amosbird)) +- Active el generador de perfiles de consulta de forma predeterminada para muestrear cada subproceso de ejecución de consultas una vez por segundo. [\#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Formato de entrada `ORC`. [\#6454](https://github.com/ClickHouse/ClickHouse/pull/6454) [\#6703](https://github.com/ClickHouse/ClickHouse/pull/6703) ([akonyaev90](https://github.com/akonyaev90)) +- Se agregaron dos nuevas funciones: `sigmoid` y `tanh` (que son útiles para aplicaciones de aprendizaje automático). [\#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Función `hasToken(haystack, token)`, `hasTokenCaseInsensitive(haystack, token)` para verificar si el token dado está en pajar. Token es una subcadena de longitud máxima entre dos caracteres ASCII no alfanuméricos (o límites de pajar). Token debe ser una cadena constante. Apoyado por la especialización de índice tokenbf\_v1. [\#6596](https://github.com/ClickHouse/ClickHouse/pull/6596), [\#6662](https://github.com/ClickHouse/ClickHouse/pull/6662) ([Vasily Nemkov](https://github.com/Enmk)) +- Nueva función `neighbor(value, offset[, default_value])`. Permite alcanzar el valor prev / next dentro de la columna en un bloque de datos. [\#5925](https://github.com/ClickHouse/ClickHouse/pull/5925) ([Acerca de Nosotros](https://github.com/alex-krash)) [6685365ab8c5b74f9650492c88a012596eb1b0c6](https://github.com/ClickHouse/ClickHouse/commit/6685365ab8c5b74f9650492c88a012596eb1b0c6) [341e2e4587a18065c2da1ca888c73389f48ce36c](https://github.com/ClickHouse/ClickHouse/commit/341e2e4587a18065c2da1ca888c73389f48ce36c) [Alexey Milovidov](https://github.com/alexey-milovidov) +- Creó una función `currentUser()`, devolver el inicio de sesión del usuario autorizado. Alias agregado `user()` por compatibilidad con MySQL. [\#6470](https://github.com/ClickHouse/ClickHouse/pull/6470) ([Acerca de Nosotros](https://github.com/alex-krash)) +- Nuevas funciones agregadas `quantilesExactInclusive` y `quantilesExactExclusive` que fueron propuestos en [\#5885](https://github.com/ClickHouse/ClickHouse/issues/5885). [\#6477](https://github.com/ClickHouse/ClickHouse/pull/6477) ([Dimarub2000](https://github.com/dimarub2000)) +- Función `bitmapRange(bitmap, range_begin, range_end)` que devuelve un nuevo conjunto con el rango especificado (no incluye el `range_end`). [\#6314](https://github.com/ClickHouse/ClickHouse/pull/6314) ([Zhichang Yu](https://github.com/yuzhichang)) +- Función `geohashesInBox(longitude_min, latitude_min, longitude_max, latitude_max, precision)` que crea una matriz de cadenas de precisión de cajas geohash que cubren el área proporcionada. [\#6127](https://github.com/ClickHouse/ClickHouse/pull/6127) ([Vasily Nemkov](https://github.com/Enmk)) +- Implementar soporte para la consulta INSERT con `Kafka` tabla. [\#6012](https://github.com/ClickHouse/ClickHouse/pull/6012) ([Ivan](https://github.com/abyss7)) +- Añadido soporte para `_partition` y `_timestamp` columnas virtuales al motor Kafka. [\#6400](https://github.com/ClickHouse/ClickHouse/pull/6400) ([Ivan](https://github.com/abyss7)) +- Posibilidad de eliminar datos confidenciales de `query_log`, registros del servidor, lista de procesos con reglas basadas en expresiones regulares. [\#5710](https://github.com/ClickHouse/ClickHouse/pull/5710) ([filimonov](https://github.com/filimonov)) + +#### Característica Experimental {#experimental-feature-2} + +- Formato de datos de entrada y salida `Template`. Permite especificar cadena de formato personalizado para entrada y salida. [\#4354](https://github.com/ClickHouse/ClickHouse/issues/4354) [\#6727](https://github.com/ClickHouse/ClickHouse/pull/6727) ([Tavplubix](https://github.com/tavplubix)) +- Implementación de `LIVE VIEW` tablas que se propusieron originalmente en [\#2898](https://github.com/ClickHouse/ClickHouse/pull/2898) elaborado en [\#3925](https://github.com/ClickHouse/ClickHouse/issues/3925), y luego actualizado en [\#5541](https://github.com/ClickHouse/ClickHouse/issues/5541). Ver [\#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) para una descripción detallada. [\#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) ([vzakaznikov](https://github.com/vzakaznikov)) [\#6425](https://github.com/ClickHouse/ClickHouse/pull/6425) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [\#6656](https://github.com/ClickHouse/ClickHouse/pull/6656) ([vzakaznikov](https://github.com/vzakaznikov)) Tenga en cuenta que `LIVE VIEW` característica puede ser eliminado en las próximas versiones. + +#### Corrección de errores {#bug-fix-8} + +- Esta versión también contiene todas las correcciones de errores de 19.13 y 19.11. +- Corregir la falla de segmentación cuando la tabla tiene índices de omisión y se produce una fusión vertical. [\#6723](https://github.com/ClickHouse/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) +- Corregir TTL por columna con valores predeterminados de columna no triviales. Anteriormente en caso de fuerza TTL se fusionan con `OPTIMIZE ... FINAL` consulta, los valores caducados se reemplazaron por valores predeterminados de tipo en lugar de valores predeterminados de columna especificados por el usuario. [\#6796](https://github.com/ClickHouse/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) +- Solucione el problema de duplicación de mensajes de Kafka en el reinicio normal del servidor. [\#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +- Se corrigió un bucle infinito al leer mensajes de Kafka. No pausar / reanudar el consumidor en la suscripción en absoluto; de lo contrario, puede pausarse indefinidamente en algunos escenarios. [\#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([Ivan](https://github.com/abyss7)) +- Fijar `Key expression contains comparison between inconvertible types` excepción en `bitmapContains` función. [\#6136](https://github.com/ClickHouse/ClickHouse/issues/6136) [\#6146](https://github.com/ClickHouse/ClickHouse/issues/6146) [\#6156](https://github.com/ClickHouse/ClickHouse/pull/6156) ([Dimarub2000](https://github.com/dimarub2000)) +- Reparar segfault con habilitado `optimize_skip_unused_shards` y falta la clave de fragmentación. [\#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([Anton Popov](https://github.com/CurtizJ)) +- Se corrigió el código incorrecto en las mutaciones que pueden conducir a la corrupción de la memoria. segfault fijo con lectura de la dirección `0x14c0` que puede happed debido a concurrente `DROP TABLE` y `SELECT` de `system.parts` o `system.parts_columns`. Condición de carrera fija en la preparación de consultas de mutación. Estancamiento fijo causado por `OPTIMIZE` de tablas replicadas y operaciones de modificación simultáneas como ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se eliminó el registro detallado adicional en la interfaz MySQL [\#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Devuelve la capacidad de analizar la configuración booleana de ‘true’ y ‘false’ en el archivo de configuración. [\#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) +- Fix crash en `quantile` y `median` función sobre `Nullable(Decimal128)`. [\#6378](https://github.com/ClickHouse/ClickHouse/pull/6378) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el posible resultado incompleto devuelto por `SELECT` consulta con `WHERE` condición en la clave principal contenía conversión a tipo flotante. Fue causado por una comprobación incorrecta de la monotonía en `toFloat` función. [\#6248](https://github.com/ClickHouse/ClickHouse/issues/6248) [\#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([Dimarub2000](https://github.com/dimarub2000)) +- Comprobar `max_expanded_ast_elements` establecimiento de mutaciones. Mutaciones claras después de `TRUNCATE TABLE`. [\#6205](https://github.com/ClickHouse/ClickHouse/pull/6205) ([Invierno Zhang](https://github.com/zhang2014)) +- Repare los resultados de JOIN para las columnas clave cuando se usa con `join_use_nulls`. Adjunte valores nulos en lugar de valores predeterminados de columnas. [\#6249](https://github.com/ClickHouse/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) +- Solución para índices de salto con combinación vertical y alteración. Solución para `Bad size of marks file` salvedad. [\#6594](https://github.com/ClickHouse/ClickHouse/issues/6594) [\#6713](https://github.com/ClickHouse/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) +- Arreglar accidente raro en `ALTER MODIFY COLUMN` y fusión vertical cuando una de las partes fusionadas / alteradas está vacía (0 filas) [\#6746](https://github.com/ClickHouse/ClickHouse/issues/6746) [\#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) +- Corregido error en la conversión de `LowCardinality` tipos en `AggregateFunctionFactory`. Esto corrige [\#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [\#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir el comportamiento incorrecto y posibles fallas en `topK` y `topKWeighted` funciones agregadas. [\#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) +- Código inseguro fijo alrededor `getIdentifier` función. [\#6401](https://github.com/ClickHouse/ClickHouse/issues/6401) [\#6409](https://github.com/ClickHouse/ClickHouse/pull/6409) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió un error en el protocolo de cable MySQL (se usa mientras se conecta al cliente MySQL de ClickHouse). Causado por el desbordamiento del búfer de pila en `PacketPayloadWriteBuffer`. [\#6212](https://github.com/ClickHouse/ClickHouse/pull/6212) ([Yuriy Baranov](https://github.com/yurriy)) +- Pérdida de memoria fija en `bitmapSubsetInRange` función. [\#6819](https://github.com/ClickHouse/ClickHouse/pull/6819) ([Zhichang Yu](https://github.com/yuzhichang)) +- Corregir un error raro cuando la mutación se ejecuta después del cambio de granularidad. [\#6816](https://github.com/ClickHouse/ClickHouse/pull/6816) ([alesapin](https://github.com/alesapin)) +- Permitir mensaje protobuf con todos los campos de forma predeterminada. [\#6132](https://github.com/ClickHouse/ClickHouse/pull/6132) ([Vitaly Baranov](https://github.com/vitlibar)) +- Resolver un error con `nullIf` función cuando enviamos un `NULL` en el segundo argumento. [\#6446](https://github.com/ClickHouse/ClickHouse/pull/6446) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Corregir un error raro con la asignación / desasignación de memoria incorrecta en diccionarios complejos de caché de claves con campos de cadena que conduce a un consumo infinito de memoria (parece una pérdida de memoria). El error se reproduce cuando el tamaño de la cadena era una potencia de dos a partir de ocho (8, 16, 32, etc.). [\#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) +- Se corrigió la codificación Gorilla en pequeñas secuencias que causaron una excepción `Cannot write after end of buffer`. [\#6398](https://github.com/ClickHouse/ClickHouse/issues/6398) [\#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Vasily Nemkov](https://github.com/Enmk)) +- Permitir el uso de tipos no anulables en JOINs con `join_use_nulls` permitir. [\#6705](https://github.com/ClickHouse/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) +- Desactivar `Poco::AbstractConfiguration` sustituciones en consulta en `clickhouse-client`. [\#6706](https://github.com/ClickHouse/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evitar el punto muerto en `REPLACE PARTITION`. [\#6677](https://github.com/ClickHouse/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Utilizar `arrayReduce` para argumentos constantes puede conducir a segfault. [\#6242](https://github.com/ClickHouse/ClickHouse/issues/6242) [\#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir partes inconsistentes que pueden aparecer si la réplica se restauró después de `DROP PARTITION`. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([Tavplubix](https://github.com/tavplubix)) +- Cuelgue fijo adentro `JSONExtractRaw` función. [\#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [\#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir un error con la serialización y agregación de índices de omisión incorrectos con granularidad adaptativa. [\#6594](https://github.com/ClickHouse/ClickHouse/issues/6594). [\#6748](https://github.com/ClickHouse/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) +- Fijar `WITH ROLLUP` y `WITH CUBE` modificadores de `GROUP BY` con agregación de dos niveles. [\#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir un error al escribir marcas de índices secundarios con granularidad adaptativa. [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) +- Corregir el orden de inicialización durante el inicio del servidor. Ya `StorageMergeTree::background_task_handle` se inicializa en `startup()` el `MergeTreeBlockOutputStream::write()` puede intentar usarlo antes de la inicialización. Solo verifique si está inicializado. [\#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) +- Borrar el búfer de datos de la operación de lectura anterior que se completó con un error. [\#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) +- Se corrigió un error al habilitar la granularidad adaptativa al crear una nueva réplica para la tabla Replicated\*MergeTree. [\#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [\#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) +- Se corrigió un posible bloqueo durante el inicio del servidor en caso de que ocurriera una excepción en `libunwind` durante la excepción en el acceso a `ThreadStatus` estructura. [\#6456](https://github.com/ClickHouse/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Fix crash en `yandexConsistentHash` función. Encontrado por prueba de fuzz. [\#6304](https://github.com/ClickHouse/ClickHouse/issues/6304) [\#6305](https://github.com/ClickHouse/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la posibilidad de colgar consultas cuando el servidor está sobrecargado y el grupo de subprocesos global está casi lleno. Esto tiene una mayor probabilidad de ocurrir en clústeres con una gran cantidad de fragmentos (cientos), porque las consultas distribuidas asignan un hilo por conexión a cada fragmento. Por ejemplo, este problema puede reproducirse si un clúster de 330 fragmentos está procesando 30 consultas distribuidas simultáneas. Este problema afecta a todas las versiones a partir de 19.2. [\#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Lógica fija de `arrayEnumerateUniqRanked` función. [\#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir segfault al decodificar la tabla de símbolos. [\#6603](https://github.com/ClickHouse/ClickHouse/pull/6603) ([Amos pájaro](https://github.com/amosbird)) +- Se corrigió una excepción irrelevante en el elenco de `LowCardinality(Nullable)` to not-Nullable column in case if it doesn't contain Nulls (e.g. in query like `SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String)`. [\#6094](https://github.com/ClickHouse/ClickHouse/issues/6094) [\#6119](https://github.com/ClickHouse/ClickHouse/pull/6119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se eliminó la cita adicional de la descripción en `system.settings` tabla. [\#6696](https://github.com/ClickHouse/ClickHouse/issues/6696) [\#6699](https://github.com/ClickHouse/ClickHouse/pull/6699) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite un posible punto muerto en `TRUNCATE` de la tabla replicada. [\#6695](https://github.com/ClickHouse/ClickHouse/pull/6695) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir la lectura en orden de clave de clasificación. [\#6189](https://github.com/ClickHouse/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) +- Fijar `ALTER TABLE ... UPDATE` consulta para tablas con `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Corregir error abierto por [\#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) (desde 19.4.0). Se reproduce en consultas a tablas distribuidas sobre tablas MergeTree cuando no consultamos ninguna columnas (`SELECT 1`). [\#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) +- Se corrigió el desbordamiento en la división de enteros de tipo con signo a tipo sin signo. El comportamiento fue exactamente como en el lenguaje C o C ++ (reglas de promoción enteras) que puede ser sorprendente. Tenga en cuenta que el desbordamiento aún es posible cuando se divide un número firmado grande por un número sin signo grande o viceversa (pero ese caso es menos habitual). El problema existía en todas las versiones del servidor. [\#6214](https://github.com/ClickHouse/ClickHouse/issues/6214) [\#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Limite el tiempo máximo de suspensión para el estrangulamiento cuando `max_execution_speed` o `max_execution_speed_bytes` se establece. Se corrigieron errores falsos como `Estimated query execution time (inf seconds) is too long`. [\#5547](https://github.com/ClickHouse/ClickHouse/issues/5547) [\#6232](https://github.com/ClickHouse/ClickHouse/pull/6232) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigieron problemas sobre el uso `MATERIALIZED` columnas y alias en `MaterializedView`. [\#448](https://github.com/ClickHouse/ClickHouse/issues/448) [\#3484](https://github.com/ClickHouse/ClickHouse/issues/3484) [\#3450](https://github.com/ClickHouse/ClickHouse/issues/3450) [\#2878](https://github.com/ClickHouse/ClickHouse/issues/2878) [\#2285](https://github.com/ClickHouse/ClickHouse/issues/2285) [\#3796](https://github.com/ClickHouse/ClickHouse/pull/3796) ([Amos pájaro](https://github.com/amosbird)) [\#6316](https://github.com/ClickHouse/ClickHouse/pull/6316) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `FormatFactory` comportamiento para flujos de entrada que no se implementan como procesador. [\#6495](https://github.com/ClickHouse/ClickHouse/pull/6495) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Error tipográfico fijo. [\#6631](https://github.com/ClickHouse/ClickHouse/pull/6631) ([Acerca de Nosotros](https://github.com/alexryndin)) +- Typo en el mensaje de error (es -\> are ). [\#6839](https://github.com/ClickHouse/ClickHouse/pull/6839) ([Denis Zhuravlev](https://github.com/den-crane)) +- Se corrigió el error al analizar la lista de columnas de la cadena si el tipo contenía una coma (este problema era relevante para `File`, `URL`, `HDFS` almacenamiento) [\#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [\#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([Dimarub2000](https://github.com/dimarub2000)) + +#### Corrección de seguridad {#security-fix} + +- Esta versión también contiene todas las correcciones de seguridad de errores de 19.13 y 19.11. +- Se corrigió la posibilidad de que una consulta fabricada causara un bloqueo del servidor debido al desbordamiento de la pila en el analizador SQL. Se corrigió la posibilidad de desbordamiento de pila en las tablas Merge y Distributed, las vistas materializadas y las condiciones para la seguridad a nivel de fila que implicaban subconsultas. [\#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora {#improvement-3} + +- Correcta implementación de la lógica ternaria para `AND/OR`. [\#6048](https://github.com/ClickHouse/ClickHouse/pull/6048) ([Alejandro Kazakov](https://github.com/Akazz)) +- Ahora los valores y filas con TTL caducado se eliminarán después `OPTIMIZE ... FINAL` query from old parts without TTL infos or with outdated TTL infos, e.g. after `ALTER ... MODIFY TTL` consulta. Consultas añadidas `SYSTEM STOP/START TTL MERGES` para no permitir / permitir asignar fusiones con TTL y filtrar valores caducados en todas las fusiones. [\#6274](https://github.com/ClickHouse/ClickHouse/pull/6274) ([Anton Popov](https://github.com/CurtizJ)) +- Posibilidad de cambiar la ubicación del archivo de historial de ClickHouse para el cliente usando `CLICKHOUSE_HISTORY_FILE` envío. [\#6840](https://github.com/ClickHouse/ClickHouse/pull/6840) ([filimonov](https://github.com/filimonov)) +- Quitar `dry_run` bandera de `InterpreterSelectQuery`. … [\#6375](https://github.com/ClickHouse/ClickHouse/pull/6375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Apoyo `ASOF JOIN` con `ON` apartado. [\#6211](https://github.com/ClickHouse/ClickHouse/pull/6211) ([Artem Zuikov](https://github.com/4ertus2)) +- Mejor soporte de índices de omisión para mutaciones y replicación. Soporte para `MATERIALIZE/CLEAR INDEX ... IN PARTITION` consulta. `UPDATE x = x` vuelve a calcular todos los índices que usan la columna `x`. [\#5053](https://github.com/ClickHouse/ClickHouse/pull/5053) ([Nikita Vasilev](https://github.com/nikvas0)) +- Permitir a `ATTACH` vistas en vivo (por ejemplo, al iniciar el servidor) independientemente de `allow_experimental_live_view` configuración. [\#6754](https://github.com/ClickHouse/ClickHouse/pull/6754) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Para los seguimientos de pila recopilados por el generador de perfiles de consultas, no incluya los marcos de pila generados por el propio generador de perfiles de consultas. [\#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora funciones de tabla `values`, `file`, `url`, `hdfs` tienen soporte para columnas ALIAS. [\#6255](https://github.com/ClickHouse/ClickHouse/pull/6255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Lance una excepción si `config.d` file no tiene el elemento raíz correspondiente como el archivo de configuración. [\#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([Dimarub2000](https://github.com/dimarub2000)) +- Imprimir información adicional en el mensaje de excepción para `no space left on device`. [\#6182](https://github.com/ClickHouse/ClickHouse/issues/6182), [\#6252](https://github.com/ClickHouse/ClickHouse/issues/6252) [\#6352](https://github.com/ClickHouse/ClickHouse/pull/6352) ([Tavplubix](https://github.com/tavplubix)) +- Al determinar fragmentos de un `Distributed` debe ser cubierto por una consulta de lectura (para `optimize_skip_unused_shards` = 1) ClickHouse ahora verifica las condiciones de ambos `prewhere` y `where` cláusulas de la instrucción select. [\#6521](https://github.com/ClickHouse/ClickHouse/pull/6521) ([Alejandro Kazakov](https://github.com/Akazz)) +- Permitir `SIMDJSON` para máquinas sin AVX2 pero con sistema de instrucciones SSE 4.2 y PCLMUL. [\#6285](https://github.com/ClickHouse/ClickHouse/issues/6285) [\#6320](https://github.com/ClickHouse/ClickHouse/pull/6320) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- ClickHouse puede funcionar en sistemas de archivos sin `O_DIRECT` soporte (como ZFS y BtrFS) sin afinación adicional. [\#4449](https://github.com/ClickHouse/ClickHouse/issues/4449) [\#6730](https://github.com/ClickHouse/ClickHouse/pull/6730) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte push down predicado para la subconsulta final. [\#6120](https://github.com/ClickHouse/ClickHouse/pull/6120) ([Método de codificación de datos:](https://github.com/TCeason)) [\#6162](https://github.com/ClickHouse/ClickHouse/pull/6162) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor `JOIN ON` extracción de llaves [\#6131](https://github.com/ClickHouse/ClickHouse/pull/6131) ([Artem Zuikov](https://github.com/4ertus2)) +- Se ha actualizado `SIMDJSON`. [\#6285](https://github.com/ClickHouse/ClickHouse/issues/6285). [\#6306](https://github.com/ClickHouse/ClickHouse/pull/6306) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Optimizar la selección de la columna más pequeña para `SELECT count()` consulta. [\#6344](https://github.com/ClickHouse/ClickHouse/pull/6344) ([Amos pájaro](https://github.com/amosbird)) +- Añadir `strict` parámetro en `windowFunnel()`. Cuando el `strict` se establece, el `windowFunnel()` aplica condiciones sólo para los valores únicos. [\#6548](https://github.com/ClickHouse/ClickHouse/pull/6548) ([chimbab](https://github.com/achimbab)) +- Interfaz más segura de `mysqlxx::Pool`. [\#6150](https://github.com/ClickHouse/ClickHouse/pull/6150) ([avasiliev](https://github.com/avasiliev)) +- Opciones de tamaño de línea al ejecutar con `--help` opción ahora se corresponde con el tamaño del terminal. [\#6590](https://github.com/ClickHouse/ClickHouse/pull/6590) ([Dimarub2000](https://github.com/dimarub2000)) +- Desactivar “read in order” optimización para la agregación sin claves. [\#6599](https://github.com/ClickHouse/ClickHouse/pull/6599) ([Anton Popov](https://github.com/CurtizJ)) +- Código de estado HTTP para `INCORRECT_DATA` y `TYPE_MISMATCH` los códigos de error se cambiaron de forma predeterminada `500 Internal Server Error` a `400 Bad Request`. [\#6271](https://github.com/ClickHouse/ClickHouse/pull/6271) ([Alejandro Rodin](https://github.com/a-rodin)) +- Mover un objeto de unión desde `ExpressionAction` en `AnalyzedJoin`. `ExpressionAnalyzer` y `ExpressionAction` no sé `Join` clase más. Su lógica está oculta por `AnalyzedJoin` iface. [\#6801](https://github.com/ClickHouse/ClickHouse/pull/6801) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió un posible interbloqueo de consultas distribuidas cuando uno de los fragmentos es localhost pero la consulta se envía a través de una conexión de red. [\#6759](https://github.com/ClickHouse/ClickHouse/pull/6759) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Semántica cambiada de varias tablas `RENAME` para evitar posibles interbloqueos. [\#6757](https://github.com/ClickHouse/ClickHouse/issues/6757). [\#6756](https://github.com/ClickHouse/ClickHouse/pull/6756) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Servidor de compatibilidad MySQL reescrito para evitar la carga completa de paquetes de carga en la memoria. Disminución del consumo de memoria para cada conexión a aproximadamente `2 * DBMS_DEFAULT_BUFFER_SIZE` (búferes de lectura/escritura). [\#5811](https://github.com/ClickHouse/ClickHouse/pull/5811) ([Yuriy Baranov](https://github.com/yurriy)) +- Mueva la lógica de interpretación de alias AST fuera del analizador que no tiene que saber nada sobre la semántica de consultas. [\#6108](https://github.com/ClickHouse/ClickHouse/pull/6108) ([Artem Zuikov](https://github.com/4ertus2)) +- Análisis ligeramente más seguro de `NamesAndTypesList`. [\#6408](https://github.com/ClickHouse/ClickHouse/issues/6408). [\#6410](https://github.com/ClickHouse/ClickHouse/pull/6410) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `clickhouse-copier`: Permitir el uso `where_condition` de configuración con `partition_key` alias en la consulta para verificar la existencia de la partición (Anteriormente se usaba solo para leer consultas de datos). [\#6577](https://github.com/ClickHouse/ClickHouse/pull/6577) ([propulsor](https://github.com/proller)) +- Añadido argumento de mensaje opcional en `throwIf`. ([\#5772](https://github.com/ClickHouse/ClickHouse/issues/5772)) [\#6329](https://github.com/ClickHouse/ClickHouse/pull/6329) ([Vdimir](https://github.com/Vdimir)) +- La excepción del servidor obtenida al enviar datos de inserción ahora también se está procesando en el cliente. [\#5891](https://github.com/ClickHouse/ClickHouse/issues/5891) [\#6711](https://github.com/ClickHouse/ClickHouse/pull/6711) ([Dimarub2000](https://github.com/dimarub2000)) +- Se agregó una métrica `DistributedFilesToInsert` que muestra el número total de archivos en el sistema de archivos que se seleccionan para enviar a servidores remotos mediante tablas distribuidas. El número se suma en todos los fragmentos. [\#6600](https://github.com/ClickHouse/ClickHouse/pull/6600) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mover la mayoría de JOINs preparar la lógica de `ExpressionAction/ExpressionAnalyzer` a `AnalyzedJoin`. [\#6785](https://github.com/ClickHouse/ClickHouse/pull/6785) ([Artem Zuikov](https://github.com/4ertus2)) +- Reparar TSan [advertencia](https://clickhouse-test-reports.s3.yandex.net/6399/c1c1d1daa98e199e620766f1bd06a5921050a00d/functional_stateful_tests_(thread).html) ‘lock-order-inversion’. [\#6740](https://github.com/ClickHouse/ClickHouse/pull/6740) ([Vasily Nemkov](https://github.com/Enmk)) +- Mejores mensajes de información sobre la falta de capacidades de Linux. Registro de errores fatales con “fatal” nivel, que hará que sea más fácil de encontrar en `system.text_log`. [\#6441](https://github.com/ClickHouse/ClickHouse/pull/6441) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cuando se habilita el volcado de datos temporales en el disco para restringir el uso de memoria durante `GROUP BY`, `ORDER BY`, no comprobó el espacio de disco libre. La corrección agrega una nueva configuración `min_free_disk_space`, cuando el espacio de disco libre es más pequeño que el umbral, la consulta se detendrá y lanzará `ErrorCodes::NOT_ENOUGH_SPACE`. [\#6678](https://github.com/ClickHouse/ClickHouse/pull/6678) ([Weiqing Xu](https://github.com/weiqxu)) [\#6691](https://github.com/ClickHouse/ClickHouse/pull/6691) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Eliminado rwlock recursivo por hilo. No tiene sentido, porque los hilos se reutilizan entre consultas. `SELECT` la consulta puede adquirir un bloqueo en un hilo, mantener un bloqueo de otro hilo y salir del primer hilo. Al mismo tiempo, el primer hilo puede ser reutilizado por `DROP` consulta. Esto conducirá a falso “Attempt to acquire exclusive lock recursively” mensaje. [\#6771](https://github.com/ClickHouse/ClickHouse/pull/6771) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Dividir `ExpressionAnalyzer.appendJoin()`. Preparar un lugar en `ExpressionAnalyzer` para `MergeJoin`. [\#6524](https://github.com/ClickHouse/ClickHouse/pull/6524) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `mysql_native_password` complemento de autenticación al servidor de compatibilidad MySQL. [\#6194](https://github.com/ClickHouse/ClickHouse/pull/6194) ([Yuriy Baranov](https://github.com/yurriy)) +- Menos número de `clock_gettime` llamadas; compatibilidad ABI fija entre depuración / liberación en `Allocator` (problema insignificante). [\#6197](https://github.com/ClickHouse/ClickHouse/pull/6197) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mover `collectUsedColumns` de `ExpressionAnalyzer` a `SyntaxAnalyzer`. `SyntaxAnalyzer` hacer `required_source_columns` sí mismo ahora. [\#6416](https://github.com/ClickHouse/ClickHouse/pull/6416) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir ajuste `joined_subquery_requires_alias` para requerir alias para subselecciones y funciones de tabla en `FROM` that more than one table is present (i.e. queries with JOINs). [\#6733](https://github.com/ClickHouse/ClickHouse/pull/6733) ([Artem Zuikov](https://github.com/4ertus2)) +- Extraer `GetAggregatesVisitor` clase de `ExpressionAnalyzer`. [\#6458](https://github.com/ClickHouse/ClickHouse/pull/6458) ([Artem Zuikov](https://github.com/4ertus2)) +- `system.query_log`: cambiar el tipo de datos de `type` columna a `Enum`. [\#6265](https://github.com/ClickHouse/ClickHouse/pull/6265) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Enlace estático de `sha256_password` complemento de autenticación. [\#6512](https://github.com/ClickHouse/ClickHouse/pull/6512) ([Yuriy Baranov](https://github.com/yurriy)) +- Evite una dependencia adicional para la configuración `compile` trabajar. En versiones anteriores, el usuario puede obtener un error como `cannot open crti.o`, `unable to find library -lc` sucesivamente. [\#6309](https://github.com/ClickHouse/ClickHouse/pull/6309) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Más validación de la entrada que puede provenir de réplica maliciosa. [\#6303](https://github.com/ClickHouse/ClickHouse/pull/6303) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora `clickhouse-obfuscator` el archivo está disponible en `clickhouse-client` paquete. En versiones anteriores estaba disponible como `clickhouse obfuscator` (con espacios en blanco). [\#5816](https://github.com/ClickHouse/ClickHouse/issues/5816) [\#6609](https://github.com/ClickHouse/ClickHouse/pull/6609) ([Dimarub2000](https://github.com/dimarub2000)) +- Fijo interbloqueo cuando tenemos al menos dos consultas que leer al menos dos tablas en orden diferente y otra consulta que realiza DDL operación en una de las mesas. Se corrigió otro punto muerto muy raro. [\#6764](https://github.com/ClickHouse/ClickHouse/pull/6764) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `os_thread_ids` columna a `system.processes` y `system.query_log` para mejores posibilidades de depuración. [\#6763](https://github.com/ClickHouse/ClickHouse/pull/6763) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Una solución para los errores de extensión PHP mysqlnd que ocurren cuando `sha256_password` se utiliza como un complemento de autenticación predeterminado (descrito en [\#6031](https://github.com/ClickHouse/ClickHouse/issues/6031)). [\#6113](https://github.com/ClickHouse/ClickHouse/pull/6113) ([Yuriy Baranov](https://github.com/yurriy)) +- Elimine el lugar innecesario con columnas de nulabilidad modificadas. [\#6693](https://github.com/ClickHouse/ClickHouse/pull/6693) ([Artem Zuikov](https://github.com/4ertus2)) +- Establecer el valor predeterminado de `queue_max_wait_ms` a cero, porque el valor actual (cinco segundos) no tiene sentido. Hay raras circunstancias en las que esta configuración tiene algún uso. Configuración añadida `replace_running_query_max_wait_ms`, `kafka_max_wait_ms` y `connection_pool_max_wait_ms` para la desambiguación. [\#6692](https://github.com/ClickHouse/ClickHouse/pull/6692) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Extraer `SelectQueryExpressionAnalyzer` de `ExpressionAnalyzer`. Mantenga el último para consultas no seleccionadas. [\#6499](https://github.com/ClickHouse/ClickHouse/pull/6499) ([Artem Zuikov](https://github.com/4ertus2)) +- Se eliminó la duplicación de formatos de entrada y salida. [\#6239](https://github.com/ClickHouse/ClickHouse/pull/6239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Permitir al usuario anular `poll_interval` y `idle_connection_timeout` configuración en la conexión. [\#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `MergeTree` ahora tiene una opción adicional `ttl_only_drop_parts` (deshabilitado por defecto) para evitar la poda parcial de las piezas, de modo que se caigan completamente cuando todas las filas de una pieza hayan caducado. [\#6191](https://github.com/ClickHouse/ClickHouse/pull/6191) ([Sergi Vladykin](https://github.com/svladykin)) +- Comprobaciones de tipo para establecer funciones de índice. Lanzar excepción si la función tiene un tipo incorrecto. Esto corrige la prueba de fuzz con UBSan. [\#6511](https://github.com/ClickHouse/ClickHouse/pull/6511) ([Nikita Vasilev](https://github.com/nikvas0)) + +#### Mejora del rendimiento {#performance-improvement-2} + +- Optimice las consultas con `ORDER BY expressions` cláusula, donde `expressions` tiene prefijo coincidente con clave de clasificación en `MergeTree` tabla. Esta optimización está controlada por `optimize_read_in_order` configuración. [\#6054](https://github.com/ClickHouse/ClickHouse/pull/6054) [\#6629](https://github.com/ClickHouse/ClickHouse/pull/6629) ([Anton Popov](https://github.com/CurtizJ)) +- Permitir el uso de múltiples hilos durante la carga y eliminación de piezas. [\#6372](https://github.com/ClickHouse/ClickHouse/issues/6372) [\#6074](https://github.com/ClickHouse/ClickHouse/issues/6074) [\#6438](https://github.com/ClickHouse/ClickHouse/pull/6438) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Variante por lotes implementada de actualizar los estados de funciones agregadas. Puede conducir a beneficios de rendimiento. [\#6435](https://github.com/ClickHouse/ClickHouse/pull/6435) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Utilizar `FastOps` biblioteca para funciones `exp`, `log`, `sigmoid`, `tanh`. FastOps es una biblioteca matemática vectorial rápido de Michael Parakhin (Yandex CTO). Rendimiento mejorado de `exp` y `log` funciona más de 6 veces. Función `exp` y `log` de `Float32` el argumento volverá `Float32` (en versiones anteriores siempre regresan `Float64`). Ahora `exp(nan)` puede volver `inf`. El resultado de `exp` y `log` las funciones pueden no ser el número representable de la máquina más cercana a la respuesta verdadera. [\#6254](https://github.com/ClickHouse/ClickHouse/pull/6254) ([alexey-milovidov](https://github.com/alexey-milovidov)) Usando la variante Danila Kutenin para hacer fastops trabajando [\#6317](https://github.com/ClickHouse/ClickHouse/pull/6317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deshabilitar la optimización de claves consecutivas para `UInt8/16`. [\#6298](https://github.com/ClickHouse/ClickHouse/pull/6298) [\#6701](https://github.com/ClickHouse/ClickHouse/pull/6701) ([akuzm](https://github.com/akuzm)) +- Rendimiento mejorado de `simdjson` biblioteca al deshacerse de la asignación dinámica en `ParsedJson::Iterator`. [\#6479](https://github.com/ClickHouse/ClickHouse/pull/6479) ([Vitaly Baranov](https://github.com/vitlibar)) +- Páginas de error previo al asignar memoria con `mmap()`. [\#6667](https://github.com/ClickHouse/ClickHouse/pull/6667) ([akuzm](https://github.com/akuzm)) +- Corregir un error de rendimiento en `Decimal` comparación. [\#6380](https://github.com/ClickHouse/ClickHouse/pull/6380) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-4} + +- Elimine el compilador (creación de instancias de la plantilla de tiempo de ejecución) porque hemos ganado su rendimiento. [\#6646](https://github.com/ClickHouse/ClickHouse/pull/6646) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó una prueba de rendimiento para mostrar la degradación del rendimiento en gcc-9 de una manera más aislada. [\#6302](https://github.com/ClickHouse/ClickHouse/pull/6302) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Función de tabla añadida `numbers_mt`, que es la versión multiproceso de `numbers`. Pruebas de rendimiento actualizadas con funciones hash. [\#6554](https://github.com/ClickHouse/ClickHouse/pull/6554) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Modo de comparación en `clickhouse-benchmark` [\#6220](https://github.com/ClickHouse/ClickHouse/issues/6220) [\#6343](https://github.com/ClickHouse/ClickHouse/pull/6343) ([Dimarub2000](https://github.com/dimarub2000)) +- Mejor esfuerzo para imprimir rastros de pila. También añadido `SIGPROF` como una señal de depuración para imprimir el seguimiento de la pila de un hilo en ejecución. [\#6529](https://github.com/ClickHouse/ClickHouse/pull/6529) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cada función en su propio archivo, parte 10. [\#6321](https://github.com/ClickHouse/ClickHouse/pull/6321) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Eliminar doble const `TABLE_IS_READ_ONLY`. [\#6566](https://github.com/ClickHouse/ClickHouse/pull/6566) ([filimonov](https://github.com/filimonov)) +- Cambios de formato para `StringHashMap` PR [\#5417](https://github.com/ClickHouse/ClickHouse/issues/5417). [\#6700](https://github.com/ClickHouse/ClickHouse/pull/6700) ([akuzm](https://github.com/akuzm)) +- Mejor subconsulta para la creación de unión en `ExpressionAnalyzer`. [\#6824](https://github.com/ClickHouse/ClickHouse/pull/6824) ([Artem Zuikov](https://github.com/4ertus2)) +- Elimine una condición redundante (encontrada por PVS Studio). [\#6775](https://github.com/ClickHouse/ClickHouse/pull/6775) ([akuzm](https://github.com/akuzm)) +- Separe la interfaz de la tabla hash para `ReverseIndex`. [\#6672](https://github.com/ClickHouse/ClickHouse/pull/6672) ([akuzm](https://github.com/akuzm)) +- Refactorización de configuraciones. [\#6689](https://github.com/ClickHouse/ClickHouse/pull/6689) ([alesapin](https://github.com/alesapin)) +- Añadir comentarios para `set` funciones de índice. [\#6319](https://github.com/ClickHouse/ClickHouse/pull/6319) ([Nikita Vasilev](https://github.com/nikvas0)) +- Aumente la puntuación de OOM en la versión de depuración en Linux. [\#6152](https://github.com/ClickHouse/ClickHouse/pull/6152) ([akuzm](https://github.com/akuzm)) +- HDFS HA ahora funciona en la compilación de depuración. [\#6650](https://github.com/ClickHouse/ClickHouse/pull/6650) ([Weiqing Xu](https://github.com/weiqxu)) +- Se agregó una prueba a `transform_query_for_external_database`. [\#6388](https://github.com/ClickHouse/ClickHouse/pull/6388) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue prueba para múltiples vistas materializadas para la tabla Kafka. [\#6509](https://github.com/ClickHouse/ClickHouse/pull/6509) ([Ivan](https://github.com/abyss7)) +- Hacer un mejor esquema de construcción. [\#6500](https://github.com/ClickHouse/ClickHouse/pull/6500) ([Ivan](https://github.com/abyss7)) +- Fijo `test_external_dictionaries` integración en caso de que se haya ejecutado bajo un usuario no root. [\#6507](https://github.com/ClickHouse/ClickHouse/pull/6507) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- El error se reproduce cuando el tamaño total de los paquetes escritos excede `DBMS_DEFAULT_BUFFER_SIZE`. [\#6204](https://github.com/ClickHouse/ClickHouse/pull/6204) ([Yuriy Baranov](https://github.com/yurriy)) +- Se agregó una prueba para `RENAME` condición de carrera de mesa [\#6752](https://github.com/ClickHouse/ClickHouse/pull/6752) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evitar la carrera de datos en Configuración en `KILL QUERY`. [\#6753](https://github.com/ClickHouse/ClickHouse/pull/6753) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir prueba de integración para el manejo de errores por un diccionario de caché. [\#6755](https://github.com/ClickHouse/ClickHouse/pull/6755) ([Vitaly Baranov](https://github.com/vitlibar)) +- Deshabilite el análisis de archivos de objetos ELF en Mac OS, porque no tiene sentido. [\#6578](https://github.com/ClickHouse/ClickHouse/pull/6578) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Intenta mejorar el generador de registro de cambios. [\#6327](https://github.com/ClickHouse/ClickHouse/pull/6327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `-Wshadow` cambiar al GCC. [\#6325](https://github.com/ClickHouse/ClickHouse/pull/6325) ([Método de codificación de datos:](https://github.com/kreuzerkrieg)) +- Eliminado el código obsoleto para `mimalloc` apoyo. [\#6715](https://github.com/ClickHouse/ClickHouse/pull/6715) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `zlib-ng` determina las capacidades de x86 y guarda esta información en variables globales. Esto se hace en la llamada defalteInit , que puede ser hecha por diferentes hilos simultáneamente. Para evitar escrituras multiproceso, hágalo al iniciar la biblioteca. [\#6141](https://github.com/ClickHouse/ClickHouse/pull/6141) ([akuzm](https://github.com/akuzm)) +- Prueba de regresión para un error que en join se corrigió en [\#5192](https://github.com/ClickHouse/ClickHouse/issues/5192). [\#6147](https://github.com/ClickHouse/ClickHouse/pull/6147) ([Bakhtiyor Ruziev](https://github.com/theruziev)) +- Informe MSan fijo. [\#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar aleteo prueba TTL. [\#6782](https://github.com/ClickHouse/ClickHouse/pull/6782) ([Anton Popov](https://github.com/CurtizJ)) +- Corregido carrera de datos falsos en `MergeTreeDataPart::is_frozen` campo. [\#6583](https://github.com/ClickHouse/ClickHouse/pull/6583) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigieron los tiempos de espera en la prueba de fuzz. En la versión anterior, se las arregló para encontrar falso bloqueo en la consulta `SELECT * FROM numbers_mt(gccMurmurHash(''))`. [\#6582](https://github.com/ClickHouse/ClickHouse/pull/6582) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron comprobaciones de depuración a `static_cast` de columnas. [\#6581](https://github.com/ClickHouse/ClickHouse/pull/6581) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte para Oracle Linux en paquetes RPM oficiales. [\#6356](https://github.com/ClickHouse/ClickHouse/issues/6356) [\#6585](https://github.com/ClickHouse/ClickHouse/pull/6585) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cambiado json perftests de `once` a `loop` tipo. [\#6536](https://github.com/ClickHouse/ClickHouse/pull/6536) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- `odbc-bridge.cpp` definir `main()` por lo que no debe ser incluido en `clickhouse-lib`. [\#6538](https://github.com/ClickHouse/ClickHouse/pull/6538) ([Películas De Sexo](https://github.com/orivej)) +- Prueba de accidente en `FULL|RIGHT JOIN` con nulos en las claves de la tabla derecha. [\#6362](https://github.com/ClickHouse/ClickHouse/pull/6362) ([Artem Zuikov](https://github.com/4ertus2)) +- Se agregó una prueba para el límite de expansión de alias por si acaso. [\#6442](https://github.com/ClickHouse/ClickHouse/pull/6442) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cambiado de `boost::filesystem` a `std::filesystem` cuando sea apropiado. [\#6253](https://github.com/ClickHouse/ClickHouse/pull/6253) [\#6385](https://github.com/ClickHouse/ClickHouse/pull/6385) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron paquetes RPM al sitio web. [\#6251](https://github.com/ClickHouse/ClickHouse/pull/6251) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir una prueba para fijo `Unknown identifier` excepción en `IN` apartado. [\#6708](https://github.com/ClickHouse/ClickHouse/pull/6708) ([Artem Zuikov](https://github.com/4ertus2)) +- Simplificar `shared_ptr_helper` porque las personas que enfrentan dificultades para entenderlo. [\#6675](https://github.com/ClickHouse/ClickHouse/pull/6675) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron pruebas de rendimiento para el códec Gorilla y DoubleDelta fijo. [\#6179](https://github.com/ClickHouse/ClickHouse/pull/6179) ([Vasily Nemkov](https://github.com/Enmk)) +- Dividir la prueba de integración `test_dictionaries` en 4 pruebas separadas. [\#6776](https://github.com/ClickHouse/ClickHouse/pull/6776) ([Vitaly Baranov](https://github.com/vitlibar)) +- Repara la advertencia de PVS-Studio en `PipelineExecutor`. [\#6777](https://github.com/ClickHouse/ClickHouse/pull/6777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Permitir usar `library` fuente del diccionario con ASan. [\#6482](https://github.com/ClickHouse/ClickHouse/pull/6482) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó la opción para generar registro de cambios a partir de una lista de relaciones públicas. [\#6350](https://github.com/ClickHouse/ClickHouse/pull/6350) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Bloquee el `TinyLog` almacenamiento al leer. [\#6226](https://github.com/ClickHouse/ClickHouse/pull/6226) ([akuzm](https://github.com/akuzm)) +- Compruebe si hay enlaces simbólicos rotos en CI. [\#6634](https://github.com/ClickHouse/ClickHouse/pull/6634) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Aumentar el tiempo de espera para “stack overflow” prueba porque puede llevar mucho tiempo en la compilación de depuración. [\#6637](https://github.com/ClickHouse/ClickHouse/pull/6637) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó un cheque para espacios en blanco dobles. [\#6643](https://github.com/ClickHouse/ClickHouse/pull/6643) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `new/delete` Seguimiento de memoria cuando se construye con desinfectantes. El seguimiento no está claro. Solo evita las excepciones de límite de memoria en las pruebas. [\#6450](https://github.com/ClickHouse/ClickHouse/pull/6450) ([Artem Zuikov](https://github.com/4ertus2)) +- Habilite la comprobación de símbolos indefinidos durante la vinculación. [\#6453](https://github.com/ClickHouse/ClickHouse/pull/6453) ([Ivan](https://github.com/abyss7)) +- Evitar la reconstrucción `hyperscan` todos los días. [\#6307](https://github.com/ClickHouse/ClickHouse/pull/6307) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Informe UBSan fijo en `ProtobufWriter`. [\#6163](https://github.com/ClickHouse/ClickHouse/pull/6163) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- No permita el uso del generador de perfiles de consultas con desinfectantes porque no es compatible. [\#6769](https://github.com/ClickHouse/ClickHouse/pull/6769) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue prueba para recargar un diccionario después de fallar por temporizador. [\#6114](https://github.com/ClickHouse/ClickHouse/pull/6114) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregir inconsistencia en `PipelineExecutor::prepareProcessor` tipo de argumento. [\#6494](https://github.com/ClickHouse/ClickHouse/pull/6494) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se agregó una prueba de URI malos. [\#6493](https://github.com/ClickHouse/ClickHouse/pull/6493) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron más cheques a `CAST` función. Esto debería obtener más información sobre la falla de segmentación en la prueba difusa. [\#6346](https://github.com/ClickHouse/ClickHouse/pull/6346) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Añadir `gcc-9` apoyo a `docker/builder` contenedor que construye la imagen localmente. [\#6333](https://github.com/ClickHouse/ClickHouse/pull/6333) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Prueba de clave primaria con `LowCardinality(String)`. [\#5044](https://github.com/ClickHouse/ClickHouse/issues/5044) [\#6219](https://github.com/ClickHouse/ClickHouse/pull/6219) ([Dimarub2000](https://github.com/dimarub2000)) +- Se corrigieron las pruebas afectadas por la impresión de rastros de pila lenta. [\#6315](https://github.com/ClickHouse/ClickHouse/pull/6315) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue un caso de prueba para el bloqueo en `groupUniqArray` fijo en [\#6029](https://github.com/ClickHouse/ClickHouse/pull/6029). [\#4402](https://github.com/ClickHouse/ClickHouse/issues/4402) [\#6129](https://github.com/ClickHouse/ClickHouse/pull/6129) ([akuzm](https://github.com/akuzm)) +- Pruebas de mutaciones de índices fijos. [\#6645](https://github.com/ClickHouse/ClickHouse/pull/6645) ([Nikita Vasilev](https://github.com/nikvas0)) +- En la prueba de rendimiento, no lea el registro de consultas para las consultas que no ejecutamos. [\#6427](https://github.com/ClickHouse/ClickHouse/pull/6427) ([akuzm](https://github.com/akuzm)) +- La vista materializada ahora se puede crear con cualquier tipo de cardinalidad baja, independientemente de la configuración sobre tipos de cardinalidad baja sospechosos. [\#6428](https://github.com/ClickHouse/ClickHouse/pull/6428) ([Olga Khvostikova](https://github.com/stavrolia)) +- Pruebas actualizadas para `send_logs_level` configuración. [\#6207](https://github.com/ClickHouse/ClickHouse/pull/6207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix build bajo gcc-8.2. [\#6196](https://github.com/ClickHouse/ClickHouse/pull/6196) ([Max Akhmedov](https://github.com/zlobober)) +- Corregir la compilación con libc ++ interno. [\#6724](https://github.com/ClickHouse/ClickHouse/pull/6724) ([Ivan](https://github.com/abyss7)) +- Reparar compilación compartida con `rdkafka` biblioteca [\#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) +- Correcciones para la compilación de Mac OS (incompleta). [\#6390](https://github.com/ClickHouse/ClickHouse/pull/6390) ([alexey-milovidov](https://github.com/alexey-milovidov)) [\#6429](https://github.com/ClickHouse/ClickHouse/pull/6429) ([Más información](https://github.com/alex-zaitsev)) +- Fijar “splitted” construir. [\#6618](https://github.com/ClickHouse/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Otras correcciones de compilación: [\#6186](https://github.com/ClickHouse/ClickHouse/pull/6186) ([Amos pájaro](https://github.com/amosbird)) [\#6486](https://github.com/ClickHouse/ClickHouse/pull/6486) [\#6348](https://github.com/ClickHouse/ClickHouse/pull/6348) ([vxider](https://github.com/Vxider)) [\#6744](https://github.com/ClickHouse/ClickHouse/pull/6744) ([Ivan](https://github.com/abyss7)) [\#6016](https://github.com/ClickHouse/ClickHouse/pull/6016) [\#6421](https://github.com/ClickHouse/ClickHouse/pull/6421) [\#6491](https://github.com/ClickHouse/ClickHouse/pull/6491) ([propulsor](https://github.com/proller)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-3} + +- Se eliminó la función de tabla raramente utilizada `catBoostPool` y almacenamiento `CatBoostPool`. Si ha utilizado esta función de tabla, escriba un correo electrónico a `clickhouse-feedback@yandex-team.com`. Tenga en cuenta que la integración CatBoost sigue siendo y será compatible. [\#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Desactivar `ANY RIGHT JOIN` y `ANY FULL JOIN` predeterminada. Establecer `any_join_distinct_right_table_keys` configuración para permitirles. [\#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [\#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) + +## Lanzamiento de ClickHouse 19.13 {#clickhouse-release-19-13} + +### Lanzamiento de ClickHouse 19.13.6.51, 2019-10-02 {#clickhouse-release-19-13-6-51-2019-10-02} + +#### Corrección de errores {#bug-fix-9} + +- Esta versión también contiene todas las correcciones de errores de 19.11.12.69. + +### Lanzamiento de ClickHouse 19.13.5.44, 2019-09-20 {#clickhouse-release-19-13-5-44-2019-09-20} + +#### Corrección de errores {#bug-fix-10} + +- Esta versión también contiene todas las correcciones de errores de 19.14.6.12. +- Se corrigió el posible estado inconsistente de la tabla mientras se ejecutaba `DROP` consulta para la tabla replicada mientras que el zookeeper no es accesible. [\#6045](https://github.com/ClickHouse/ClickHouse/issues/6045) [\#6413](https://github.com/ClickHouse/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Solución para la carrera de datos en StorageMerge [\#6717](https://github.com/ClickHouse/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir error introducido en el generador de perfiles de consultas que conduce a una recv interminable desde el socket. [\#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) +- Corregir el uso excesivo de la CPU mientras se ejecuta `JSONExtractRaw` función sobre un valor booleano. [\#6208](https://github.com/ClickHouse/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corrige la regresión mientras presiona a la vista materializada. [\#6415](https://github.com/ClickHouse/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) +- Función de la tabla `url` la vulnerabilidad permitió al atacante inyectar encabezados HTTP arbitrarios en la solicitud. Este problema fue encontrado por [Nikita Tikhomirov](https://github.com/NSTikhomirov). [\#6466](https://github.com/ClickHouse/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Arreglar inútil `AST` compruebe en Establecer índice. [\#6510](https://github.com/ClickHouse/ClickHouse/issues/6510) [\#6651](https://github.com/ClickHouse/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) +- Análisis fijo de `AggregateFunction` valores incrustados en la consulta. [\#6575](https://github.com/ClickHouse/ClickHouse/issues/6575) [\#6773](https://github.com/ClickHouse/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) +- Corregido el comportamiento incorrecto de `trim` funciones familiares. [\#6647](https://github.com/ClickHouse/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.13.4.32, 2019-09-10 {#clickhouse-release-19-13-4-32-2019-09-10} + +#### Corrección de errores {#bug-fix-11} + +- Esta versión también contiene todas las correcciones de seguridad de errores de 19.11.9.52 y 19.11.10.54. +- Carrera de datos fija en `system.parts` mesa y `ALTER` consulta. [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245) [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el encabezado no coincidente en las secuencias en caso de leer desde una tabla distribuida vacía con sample y prewhere. [\#6167](https://github.com/ClickHouse/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [\#6823](https://github.com/ClickHouse/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se corrigió el fallo al usar `IN` cláusula con una subconsulta con una tupla. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([Tavplubix](https://github.com/tavplubix)) +- Corregir el caso con los mismos nombres de columna en `GLOBAL JOIN ON` apartado. [\#6181](https://github.com/ClickHouse/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregir el bloqueo al lanzar tipos a `Decimal` que no lo soportan. Lanzar excepción en su lugar. [\#6297](https://github.com/ClickHouse/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el fallo en `extractAll()` función. [\#6644](https://github.com/ClickHouse/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) +- Transformación de consultas para `MySQL`, `ODBC`, `JDBC` funciones de la tabla ahora funciona correctamente para `SELECT WHERE` consultas con múltiples `AND` expresiones. [\#6381](https://github.com/ClickHouse/ClickHouse/issues/6381) [\#6676](https://github.com/ClickHouse/ClickHouse/pull/6676) ([Dimarub2000](https://github.com/dimarub2000)) +- Se agregaron verificaciones de declaraciones anteriores para la integración de MySQL 8. [\#6569](https://github.com/ClickHouse/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) + +#### Corrección de seguridad {#security-fix-1} + +- Corrige dos vulnerabilidades en los códecs en fase de descompresión (el usuario malicioso puede fabricar datos comprimidos que provocarán un desbordamiento del búfer en la descompresión). [\#6670](https://github.com/ClickHouse/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) + +### Lanzamiento de ClickHouse 19.13.3.26, 2019-08-22 {#clickhouse-release-19-13-3-26-2019-08-22} + +#### Corrección de errores {#bug-fix-12} + +- Fijar `ALTER TABLE ... UPDATE` consulta para tablas con `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Repare NPE al usar la cláusula IN con una subconsulta con una tupla. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([Tavplubix](https://github.com/tavplubix)) +- Se ha solucionado un problema por el que si una réplica obsoleta cobra vida, puede que aún tenga partes de datos eliminadas por DROP PARTITION. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([Tavplubix](https://github.com/tavplubix)) +- Solucionado el problema con el análisis CSV [\#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [\#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([Tavplubix](https://github.com/tavplubix)) +- Carrera de datos fija en el sistema.tabla de piezas y consulta ALTER. Esto corrige [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el código incorrecto en las mutaciones que pueden conducir a la corrupción de la memoria. segfault fijo con lectura de la dirección `0x14c0` que puede happed debido a concurrente `DROP TABLE` y `SELECT` de `system.parts` o `system.parts_columns`. Condición de carrera fija en la preparación de consultas de mutación. Estancamiento fijo causado por `OPTIMIZE` de tablas replicadas y operaciones de modificación simultáneas como ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la posible pérdida de datos después de `ALTER DELETE` consulta en la tabla con índice de omisión. [\#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [\#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) + +#### Corrección de seguridad {#security-fix-2} + +- Si el atacante tiene acceso de escritura a ZooKeeper y es capaz de ejecutar servidor personalizado disponible desde la red donde se ejecuta ClickHouse, puede crear servidor malicioso personalizado que actuará como réplica ClickHouse y registrarlo en ZooKeeper. Cuando otra réplica recuperará la parte de datos de una réplica maliciosa, puede forzar a clickhouse-server a escribir en una ruta arbitraria en el sistema de archivos. Encontrado por Eldar Zaitov, equipo de seguridad de la información en Yandex. [\#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.13.2.19, 2019-08-14 {#clickhouse-release-19-13-2-19-2019-08-14} + +#### Novedad {#new-feature-5} + +- Generador de perfiles de muestreo en el nivel de consulta. [Ejemplo](https://gist.github.com/alexey-milovidov/92758583dd41c24c360fdb8d6a4da194). [\#4247](https://github.com/ClickHouse/ClickHouse/issues/4247) ([laplab](https://github.com/laplab)) [\#6124](https://github.com/ClickHouse/ClickHouse/pull/6124) ([alexey-milovidov](https://github.com/alexey-milovidov)) [\#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) [\#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) [\#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) +- Permite especificar una lista de columnas con `COLUMNS('regexp')` expresión que funciona como una variante más sofisticada de `*` asterisco. [\#5951](https://github.com/ClickHouse/ClickHouse/pull/5951) ([Mfridental](https://github.com/mfridental)), ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `CREATE TABLE AS table_function()` es posible [\#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([Dimarub2000](https://github.com/dimarub2000)) +- El optimizador de Adam para el descenso de gradiente estocástico se usa de forma predeterminada en `stochasticLinearRegression()` y `stochasticLogisticRegression()` funciones agregadas, porque muestra buena calidad sin casi ningún ajuste. [\#6000](https://github.com/ClickHouse/ClickHouse/pull/6000) ([Quid37](https://github.com/Quid37)) +- Added functions for working with the сustom week number [\#5212](https://github.com/ClickHouse/ClickHouse/pull/5212) ([Cristina Andrés](https://github.com/andyyzh)) +- `RENAME` las consultas ahora funcionan con todos los almacenes. [\#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) +- Ahora el cliente recibe registros del servidor con cualquier nivel deseado configurando `send_logs_level` independientemente del nivel de registro especificado en la configuración del servidor. [\#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-4} + +- Configuración `input_format_defaults_for_omitted_fields` está habilitado de forma predeterminada. Las inserciones en las tablas distribuidas necesitan que esta configuración sea la misma en el clúster (debe configurarla antes de actualizar). Permite el cálculo de expresiones predeterminadas complejas para campos omitidos en `JSONEachRow` y `CSV*` formato. Debe ser el comportamiento esperado, pero puede conducir a una diferencia de rendimiento insignificante. [\#6043](https://github.com/ClickHouse/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [\#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) + +#### Experimental características {#experimental-features} + +- Nueva canalización de procesamiento de consultas. Utilizar `experimental_use_processors=1` opción para habilitarlo. Úselo para su propio problema. [\#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Corrección de errores {#bug-fix-13} + +- Kafka integración se ha fijado en esta versión. +- Fijo `DoubleDelta` codificación de `Int64` para grandes `DoubleDelta` valores mejorados `DoubleDelta` codificación de datos aleatorios para `Int32`. [\#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) +- Sobreestimación fija de `max_rows_to_read` si el ajuste `merge_tree_uniform_read_distribution` se establece en 0. [\#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora {#improvement-4} + +- Lanza una excepción si `config.d` el archivo no tiene el elemento raíz correspondiente como el archivo de configuración [\#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([Dimarub2000](https://github.com/dimarub2000)) + +#### Mejora del rendimiento {#performance-improvement-3} + +- Optimizar `count()`. Ahora usa la columna más pequeña (si es posible). [\#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos pájaro](https://github.com/amosbird)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-5} + +- Informe sobre el uso de memoria en las pruebas de rendimiento. [\#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) +- Corregir compilación con externo `libcxx` [\#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) +- Reparar compilación compartida con `rdkafka` biblioteca [\#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) + +## Lanzamiento de ClickHouse 19.11 {#clickhouse-release-19-11} + +### Lanzamiento de ClickHouse 19.11.13.74, 2019-11-01 {#clickhouse-release-19-11-13-74-2019-11-01} + +#### Corrección de errores {#bug-fix-14} + +- Se corrigió un accidente raro en `ALTER MODIFY COLUMN` y fusión vertical cuando una de las partes fusionadas / alteradas está vacía (0 filas). [\#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) +- Actualización manual de `SIMDJSON`. Esto corrige la posible inundación de archivos stderr con mensajes de diagnóstico json falsos. [\#7548](https://github.com/ClickHouse/ClickHouse/pull/7548) ([Alejandro Kazakov](https://github.com/Akazz)) +- Corregido error con `mrk` extensión de archivo para mutaciones ([alesapin](https://github.com/alesapin)) + +### Lanzamiento de ClickHouse 19.11.12.69, 2019-10-02 {#clickhouse-release-19-11-12-69-2019-10-02} + +#### Corrección de errores {#bug-fix-15} + +- Se corrigió la degradación del rendimiento del análisis de índices en claves complejas en tablas grandes. Esto corrige [\#6924](https://github.com/ClickHouse/ClickHouse/issues/6924). [\#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite SIGSEGV raro al enviar datos en tablas con motor distribuido (`Failed to send batch: file with index XXXXX is absent`). [\#7032](https://github.com/ClickHouse/ClickHouse/pull/7032) ([Azat Khuzhin](https://github.com/azat)) +- Fijar `Unknown identifier` con múltiples uniones. Esto corrige [\#5254](https://github.com/ClickHouse/ClickHouse/issues/5254). [\#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) + +### Lanzamiento de ClickHouse 19.11.11.57, 2019-09-13 {#clickhouse-release-19-11-11-57-2019-09-13} + +- Corregir el error lógico que causa segfaults al seleccionar el tema vacío de Kafka. [\#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [\#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +- Solución para la función `АrrayEnumerateUniqRanked` con matrices vacías en params. [\#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([propulsor](https://github.com/proller)) + +### Lanzamiento de ClickHouse 19.11.10.54, 2019-09-10 {#clickhouse-release-19-11-10-54-2019-09-10} + +#### Corrección de errores {#bug-fix-16} + +- Almacene los desplazamientos para los mensajes de Kafka manualmente para poder confirmarlos todos a la vez para todas las particiones. Corrige la duplicación potencial en “one consumer - many partitions” escenario. [\#6872](https://github.com/ClickHouse/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) + +### Lanzamiento de ClickHouse 19.11.9.52, 2019-09-6 {#clickhouse-release-19-11-9-52-2019-09-6} + +- Mejorar el manejo de errores en los diccionarios de caché. [\#6737](https://github.com/ClickHouse/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregido error en la función `arrayEnumerateUniqRanked`. [\#6779](https://github.com/ClickHouse/ClickHouse/pull/6779) ([propulsor](https://github.com/proller)) +- Fijar `JSONExtract` función mientras se extrae un `Tuple` de JSON. [\#6718](https://github.com/ClickHouse/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) +- Se corrigió la posible pérdida de datos después de `ALTER DELETE` consulta en la tabla con índice de omisión. [\#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [\#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) +- Prueba de rendimiento fija. [\#6392](https://github.com/ClickHouse/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Parquet: Fijar la lectura de columnas booleanos. [\#6579](https://github.com/ClickHouse/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregido el comportamiento incorrecto de `nullIf` función de argumentos constantes. [\#6518](https://github.com/ClickHouse/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [\#6580](https://github.com/ClickHouse/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Solucione el problema de duplicación de mensajes de Kafka en el reinicio normal del servidor. [\#6597](https://github.com/ClickHouse/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +- Se ha corregido un problema durante mucho tiempo `ALTER UPDATE` o `ALTER DELETE` puede evitar que se ejecuten fusiones regulares. Evite que las mutaciones se ejecuten si no hay suficientes subprocesos libres disponibles. [\#6502](https://github.com/ClickHouse/ClickHouse/issues/6502) [\#6617](https://github.com/ClickHouse/ClickHouse/pull/6617) ([Tavplubix](https://github.com/tavplubix)) +- Corregido el error con el procesamiento “timezone” en el archivo de configuración del servidor. [\#6709](https://github.com/ClickHouse/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Repara las pruebas de kafka. [\#6805](https://github.com/ClickHouse/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) + +#### Corrección de seguridad {#security-fix-3} + +- Si el atacante tiene acceso de escritura a ZooKeeper y es capaz de ejecutar servidor personalizado disponible desde la red donde se ejecuta ClickHouse, puede crear servidor malicioso personalizado que actuará como réplica ClickHouse y registrarlo en ZooKeeper. Cuando otra réplica recuperará la parte de datos de una réplica maliciosa, puede forzar a clickhouse-server a escribir en una ruta arbitraria en el sistema de archivos. Encontrado por Eldar Zaitov, equipo de seguridad de la información en Yandex. [\#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.11.8.46, 2019-08-22 {#clickhouse-release-19-11-8-46-2019-08-22} + +#### Corrección de errores {#bug-fix-17} + +- Fijar `ALTER TABLE ... UPDATE` consulta para tablas con `enable_mixed_granularity_parts=1`. [\#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +- Repare NPE al usar la cláusula IN con una subconsulta con una tupla. [\#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [\#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([Tavplubix](https://github.com/tavplubix)) +- Se ha solucionado un problema por el que si una réplica obsoleta cobra vida, puede que aún tenga partes de datos eliminadas por DROP PARTITION. [\#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [\#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([Tavplubix](https://github.com/tavplubix)) +- Solucionado el problema con el análisis CSV [\#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [\#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([Tavplubix](https://github.com/tavplubix)) +- Carrera de datos fija en el sistema.tabla de piezas y consulta ALTER. Esto corrige [\#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [\#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el código incorrecto en las mutaciones que pueden conducir a la corrupción de la memoria. segfault fijo con lectura de la dirección `0x14c0` que puede happed debido a concurrente `DROP TABLE` y `SELECT` de `system.parts` o `system.parts_columns`. Condición de carrera fija en la preparación de consultas de mutación. Estancamiento fijo causado por `OPTIMIZE` de tablas replicadas y operaciones de modificación simultáneas como ALTERs. [\#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.11.7.40, 2019-08-14 {#clickhouse-release-19-11-7-40-2019-08-14} + +#### Corrección de errores {#bug-fix-18} + +- Kafka integración se ha fijado en esta versión. +- Corregir segfault cuando se usa `arrayReduce` para argumentos constantes. [\#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijo `toFloat()` monotonicidad. [\#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([Dimarub2000](https://github.com/dimarub2000)) +- Reparar segfault con habilitado `optimize_skip_unused_shards` y falta la clave de fragmentación. [\#6384](https://github.com/ClickHouse/ClickHouse/pull/6384) ([CurtizJ](https://github.com/CurtizJ)) +- Lógica fija de `arrayEnumerateUniqRanked` función. [\#6423](https://github.com/ClickHouse/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se eliminó el registro detallado adicional del controlador MySQL. [\#6389](https://github.com/ClickHouse/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir el comportamiento incorrecto y posibles fallas en `topK` y `topKWeighted` funciones agregadas. [\#6404](https://github.com/ClickHouse/ClickHouse/pull/6404) ([CurtizJ](https://github.com/CurtizJ)) +- No exponga columnas virtuales en `system.columns` tabla. Esto es necesario para la compatibilidad con versiones anteriores. [\#6406](https://github.com/ClickHouse/ClickHouse/pull/6406) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir un error con la asignación de memoria para campos de cadena en el diccionario de caché de claves complejas. [\#6447](https://github.com/ClickHouse/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) +- Corregir un error al habilitar la granularidad adaptativa al crear una nueva réplica para `Replicated*MergeTree` tabla. [\#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) +- Corregir bucle infinito al leer los mensajes de Kafka. [\#6354](https://github.com/ClickHouse/ClickHouse/pull/6354) ([abyss7](https://github.com/abyss7)) +- Se corrigió la posibilidad de que una consulta fabricada causara un bloqueo del servidor debido al desbordamiento de la pila en el analizador SQL y la posibilidad de desbordamiento de la pila en `Merge` y `Distributed` tabla [\#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregido el error de codificación Gorilla en pequeñas secuencias. [\#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Enmk](https://github.com/Enmk)) + +#### Mejora {#improvement-5} + +- Permitir al usuario anular `poll_interval` y `idle_connection_timeout` configuración en la conexión. [\#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.11.5.28, 2019-08-05 {#clickhouse-release-19-11-5-28-2019-08-05} + +#### Corrección de errores {#bug-fix-19} + +- Se corrigió la posibilidad de colgar consultas cuando el servidor está sobrecargado. [\#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparar FPE en la función yandexConsistentHash. Esto corrige [\#6304](https://github.com/ClickHouse/ClickHouse/issues/6304). [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregido error en la conversión de `LowCardinality` tipos en `AggregateFunctionFactory`. Esto corrige [\#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [\#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir el análisis de `bool` ajustes de `true` y `false` en los archivos de configuración. [\#6278](https://github.com/ClickHouse/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) +- Corrige un error raro con encabezados de flujo incompatibles en consultas para `Distributed` mesa sobre `MergeTree` mesa cuando parte de `WHERE` se mueve a `PREWHERE`. [\#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) +- Se corrigió el desbordamiento en la división de enteros de tipo con signo a tipo sin signo. Esto corrige [\#6214](https://github.com/ClickHouse/ClickHouse/issues/6214). [\#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-5} + +- `Kafka` todavía roto. + +### Lanzamiento de ClickHouse 19.11.4.24, 2019-08-01 {#clickhouse-release-19-11-4-24-2019-08-01} + +#### Corrección de errores {#bug-fix-20} + +- Corregir un error al escribir marcas de índices secundarios con granularidad adaptativa. [\#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) +- Fijar `WITH ROLLUP` y `WITH CUBE` modificadores de `GROUP BY` con agregación de dos niveles. [\#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) +- Cuelgue fijo adentro `JSONExtractRaw` función. Fijo [\#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [\#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparar segfault en ExternalLoader :: loadOutdated (). [\#6082](https://github.com/ClickHouse/ClickHouse/pull/6082) ([Vitaly Baranov](https://github.com/vitlibar)) +- Se corrigió el caso en el que el servidor podía cerrar sockets de escucha pero no apagar y continuar sirviendo las consultas restantes. Puede terminar con dos procesos de servidor de clickhouse en ejecución. A veces, el servidor puede devolver un error `bad_function_call` para las consultas restantes. [\#6231](https://github.com/ClickHouse/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la condición inútil e incorrecta en el campo de actualización para la carga inicial de diccionarios externos a través de ODBC, MySQL, ClickHouse y HTTP. Esto corrige [\#6069](https://github.com/ClickHouse/ClickHouse/issues/6069) [\#6083](https://github.com/ClickHouse/ClickHouse/pull/6083) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió una excepción irrelevante en el elenco de `LowCardinality(Nullable)` to not-Nullable column in case if it doesn't contain Nulls (e.g. in query like `SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String)`. [\#6094](https://github.com/ClickHouse/ClickHouse/issues/6094) [\#6119](https://github.com/ClickHouse/ClickHouse/pull/6119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fijar resultado no determinista de “uniq” función agregada en casos extremos raros. El error estaba presente en todas las versiones de ClickHouse. [\#6058](https://github.com/ClickHouse/ClickHouse/pull/6058) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Segfault cuando establecemos un CIDR demasiado alto en la función `IPv6CIDRToRange`. [\#6068](https://github.com/ClickHouse/ClickHouse/pull/6068) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Se corrigió una pequeña pérdida de memoria cuando el servidor lanzaba muchas excepciones de muchos contextos diferentes. [\#6144](https://github.com/ClickHouse/ClickHouse/pull/6144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soluciona la situación cuando el consumidor se detuvo antes de la suscripción y no se reanudó después. [\#6075](https://github.com/ClickHouse/ClickHouse/pull/6075) ([Ivan](https://github.com/abyss7)) Tenga en cuenta que Kafka está roto en esta versión. +- Borrar el búfer de datos Kafka de la operación de lectura anterior que se completó con un error [\#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) Tenga en cuenta que Kafka está roto en esta versión. +- Ya `StorageMergeTree::background_task_handle` se inicializa en `startup()` el `MergeTreeBlockOutputStream::write()` puede intentar usarlo antes de la inicialización. Solo verifique si está inicializado. [\#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-6} + +- Añadido oficial `rpm` paquete. [\#5740](https://github.com/ClickHouse/ClickHouse/pull/5740) ([propulsor](https://github.com/proller)) ([alesapin](https://github.com/alesapin)) +- Añadir una habilidad para construir `.rpm` y `.tgz` paquetes con `packager` script. [\#5769](https://github.com/ClickHouse/ClickHouse/pull/5769) ([alesapin](https://github.com/alesapin)) +- Correcciones para “Arcadia” sistema de construcción. [\#6223](https://github.com/ClickHouse/ClickHouse/pull/6223) ([propulsor](https://github.com/proller)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-6} + +- `Kafka` está roto en esta versión. + +### Lanzamiento de ClickHouse 19.11.3.11, 2019-07-18 {#clickhouse-release-19-11-3-11-2019-07-18} + +#### Novedad {#new-feature-6} + +- Se agregó soporte para declaraciones preparadas. [\#5331](https://github.com/ClickHouse/ClickHouse/pull/5331/) ([Alejandro](https://github.com/sanych73)) [\#5630](https://github.com/ClickHouse/ClickHouse/pull/5630) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `DoubleDelta` y `Gorilla` codecs de columna [\#5600](https://github.com/ClickHouse/ClickHouse/pull/5600) ([Vasily Nemkov](https://github.com/Enmk)) +- Añadir `os_thread_priority` ajuste que permite controlar el “nice” valor de los subprocesos de procesamiento de consultas que utiliza el sistema operativo para ajustar la prioridad de programación dinámica. Requiere `CAP_SYS_NICE` capacidades para trabajar. Esto implementa [\#5858](https://github.com/ClickHouse/ClickHouse/issues/5858) [\#5909](https://github.com/ClickHouse/ClickHouse/pull/5909) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Implementar `_topic`, `_offset`, `_key` columnas para el motor Kafka [\#5382](https://github.com/ClickHouse/ClickHouse/pull/5382) ([Ivan](https://github.com/abyss7)) Tenga en cuenta que Kafka está roto en esta versión. +- Añadir combinador de funciones agregadas `-Resample` [\#5590](https://github.com/ClickHouse/ClickHouse/pull/5590) ([Hcz](https://github.com/hczhcz)) +- Funciones agregadas `groupArrayMovingSum(win_size)(x)` y `groupArrayMovingAvg(win_size)(x)`, que calculan la suma / avg en movimiento con o sin limitación de tamaño de ventana. [\#5595](https://github.com/ClickHouse/ClickHouse/pull/5595) ([Sistema abierto.](https://github.com/inv2004)) +- Agregar sinónimo `arrayFlatten` \<-\> `flatten` [\#5764](https://github.com/ClickHouse/ClickHouse/pull/5764) ([Hcz](https://github.com/hczhcz)) +- Función Intergate H3 `geoToH3` de Uber. [\#4724](https://github.com/ClickHouse/ClickHouse/pull/4724) ([Remen Ivan](https://github.com/BHYCHIK)) [\#5805](https://github.com/ClickHouse/ClickHouse/pull/5805) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Corrección de errores {#bug-fix-21} + +- Implementar caché DNS con actualización asincrónica. El subproceso separado resuelve todos los hosts y actualiza la caché DNS con el punto (configuración `dns_cache_update_period`). Debería ayudar, cuando la ip de los hosts cambia con frecuencia. [\#5857](https://github.com/ClickHouse/ClickHouse/pull/5857) ([Anton Popov](https://github.com/CurtizJ)) +- Arreglar segfault en `Delta` codec que afecta a columnas con valores de menos de 32 bits. El error llevó a la corrupción de memoria aleatoria. [\#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) +- Fix segfault en TTL se fusionan con columnas no físicas en el bloque. [\#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir un error raro en la comprobación de la parte con `LowCardinality` columna. Previamente `checkDataPart` siempre falla para parte con `LowCardinality` columna. [\#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) +- Evite colgar conexiones cuando el grupo de subprocesos del servidor esté lleno. Es importante para las conexiones desde `remote` función de tabla o conexiones a un fragmento sin réplicas cuando hay un tiempo de espera de conexión largo. Esto corrige [\#5878](https://github.com/ClickHouse/ClickHouse/issues/5878) [\#5881](https://github.com/ClickHouse/ClickHouse/pull/5881) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte para argumentos constantes para `evalMLModel` función. Esto corrige [\#5817](https://github.com/ClickHouse/ClickHouse/issues/5817) [\#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha solucionado el problema cuando ClickHouse determina la zona horaria predeterminada como `UCT` en lugar de `UTC`. Esto corrige [\#5804](https://github.com/ClickHouse/ClickHouse/issues/5804). [\#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijo desbordamiento del búfer en `visitParamExtractRaw`. Esto corrige [\#5901](https://github.com/ClickHouse/ClickHouse/issues/5901) [\#5902](https://github.com/ClickHouse/ClickHouse/pull/5902) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora distribuido `DROP/ALTER/TRUNCATE/OPTIMIZE ON CLUSTER` las consultas se ejecutarán directamente en la réplica del líder. [\#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) +- Fijar `coalesce` para `ColumnConst` con `ColumnNullable` + cambios relacionados. [\#5755](https://github.com/ClickHouse/ClickHouse/pull/5755) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar el `ReadBufferFromKafkaConsumer` para que siga leyendo nuevos mensajes después `commit()` incluso si estaba estancado antes [\#5852](https://github.com/ClickHouse/ClickHouse/pull/5852) ([Ivan](https://github.com/abyss7)) +- Fijar `FULL` y `RIGHT` Resultados de JOIN al unirse en `Nullable` teclas en la mesa derecha. [\#5859](https://github.com/ClickHouse/ClickHouse/pull/5859) ([Artem Zuikov](https://github.com/4ertus2)) +- Posible solución de sueño infinito de consultas de baja prioridad. [\#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir la condición de carrera, lo que hace que algunas consultas no aparezcan en query\_log después `SYSTEM FLUSH LOGS` consulta. [\#5456](https://github.com/ClickHouse/ClickHouse/issues/5456) [\#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) +- Fijo `heap-use-after-free` ASan advertencia en ClusterCopier causada por el reloj que intenta usar el objeto de copiadora ya eliminado. [\#5871](https://github.com/ClickHouse/ClickHouse/pull/5871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregido mal `StringRef` puntero devuelto por algunas implementaciones de `IColumn::deserializeAndInsertFromArena`. Este error afectó solo a las pruebas unitarias. [\#5973](https://github.com/ClickHouse/ClickHouse/pull/5973) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Evite que la matriz de origen y la matriz intermedia se unan a columnas de enmascaramiento de columnas del mismo nombre. [\#5941](https://github.com/ClickHouse/ClickHouse/pull/5941) ([Artem Zuikov](https://github.com/4ertus2)) +- Solucione la inserción y seleccione la consulta al motor MySQL con citas de identificador de estilo MySQL. [\#5704](https://github.com/ClickHouse/ClickHouse/pull/5704) ([Invierno Zhang](https://github.com/zhang2014)) +- Ahora `CHECK TABLE` consulta puede trabajar con la familia del motor MergeTree. Devuelve el estado de verificación y el mensaje si hay alguno para cada parte (o archivo en caso de motores más simples). Además, corrija el error en la búsqueda de una parte rota. [\#5865](https://github.com/ClickHouse/ClickHouse/pull/5865) ([alesapin](https://github.com/alesapin)) +- Reparar el tiempo de ejecución SPLIT\_SHARED\_LIBRARIES [\#5793](https://github.com/ClickHouse/ClickHouse/pull/5793) ([Más información](https://github.com/danlark1)) +- Inicialización de zona horaria fija cuando `/etc/localtime` es un enlace simbólico relativo como `../usr/share/zoneinfo/Europe/Moscow` [\#5922](https://github.com/ClickHouse/ClickHouse/pull/5922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- clickhouse-copiadora: Solucionar el uso-después de la libertad en el apagado [\#5752](https://github.com/ClickHouse/ClickHouse/pull/5752) ([propulsor](https://github.com/proller)) +- Actualizar `simdjson`. Se corrigió el problema de que algunos JSON no válidos con cero bytes analizaran correctamente. [\#5938](https://github.com/ClickHouse/ClickHouse/pull/5938) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparar el cierre de SystemLogs [\#5802](https://github.com/ClickHouse/ClickHouse/pull/5802) ([Anton Popov](https://github.com/CurtizJ)) +- Arreglar colgando cuando la condición en invalidate\_query depende de un diccionario. [\#6011](https://github.com/ClickHouse/ClickHouse/pull/6011) ([Vitaly Baranov](https://github.com/vitlibar)) + +#### Mejora {#improvement-6} + +- Permitir direcciones no resolubles en la configuración del clúster. Se considerarán no disponibles y se intentarán resolver en cada intento de conexión. Esto es especialmente útil para Kubernetes. Esto corrige [\#5714](https://github.com/ClickHouse/ClickHouse/issues/5714) [\#5924](https://github.com/ClickHouse/ClickHouse/pull/5924) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cierre las conexiones TCP inactivas (con un tiempo de espera de una hora por defecto). Esto es especialmente importante para clústeres grandes con múltiples tablas distribuidas en cada servidor, ya que cada servidor puede mantener un grupo de conexiones con cualquier otro servidor, y después de la concurrencia de consultas pico, las conexiones se detendrán. Esto corrige [\#5879](https://github.com/ClickHouse/ClickHouse/issues/5879) [\#5880](https://github.com/ClickHouse/ClickHouse/pull/5880) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor calidad de `topK` función. Se ha cambiado el comportamiento del conjunto SavingSpace para eliminar el último elemento si el nuevo elemento tiene un peso mayor. [\#5833](https://github.com/ClickHouse/ClickHouse/issues/5833) [\#5850](https://github.com/ClickHouse/ClickHouse/pull/5850) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Las funciones de URL para trabajar con dominios ahora pueden funcionar para URL incompletas sin esquema [\#5725](https://github.com/ClickHouse/ClickHouse/pull/5725) ([alesapin](https://github.com/alesapin)) +- Sumas de comprobación añadidas al `system.parts_columns` tabla. [\#5874](https://github.com/ClickHouse/ClickHouse/pull/5874) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Añadir `Enum` tipo de datos como un sinónimo para `Enum8` o `Enum16`. [\#5886](https://github.com/ClickHouse/ClickHouse/pull/5886) ([Dimarub2000](https://github.com/dimarub2000)) +- Variante de transposición de bits completa para `T64` códec. Podría conducir a una mejor compresión con `zstd`. [\#5742](https://github.com/ClickHouse/ClickHouse/pull/5742) ([Artem Zuikov](https://github.com/4ertus2)) +- Condición en `startsWith` la función ahora puede utilizar la clave principal. Esto corrige [\#5310](https://github.com/ClickHouse/ClickHouse/issues/5310) y [\#5882](https://github.com/ClickHouse/ClickHouse/issues/5882) [\#5919](https://github.com/ClickHouse/ClickHouse/pull/5919) ([Dimarub2000](https://github.com/dimarub2000)) +- Permitir usar `clickhouse-copier` con la topología de clúster de replicación cruzada permitiendo el nombre de base de datos vacío. [\#5745](https://github.com/ClickHouse/ClickHouse/pull/5745) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) +- Utilizar `UTC` como zona horaria predeterminada en un sistema sin `tzdata` (e.g. bare Docker container). Before this patch, error message `Could not determine local time zone` se imprimió y el servidor o el cliente se negó a iniciar. [\#5827](https://github.com/ClickHouse/ClickHouse/pull/5827) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte devuelto para argumento de coma flotante en función `quantileTiming` para la compatibilidad con versiones anteriores. [\#5911](https://github.com/ClickHouse/ClickHouse/pull/5911) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mostrar qué tabla falta la columna en los mensajes de error. [\#5768](https://github.com/ClickHouse/ClickHouse/pull/5768) ([Ivan](https://github.com/abyss7)) +- No permitir la consulta de ejecución con el mismo query\_id por varios usuarios [\#5430](https://github.com/ClickHouse/ClickHouse/pull/5430) ([propulsor](https://github.com/proller)) +- Código más robusto para enviar métricas a Graphite. Funcionará incluso durante largos múltiples `RENAME TABLE` operación. [\#5875](https://github.com/ClickHouse/ClickHouse/pull/5875) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se mostrarán mensajes de error más informativos cuando ThreadPool no pueda programar una tarea para su ejecución. Esto corrige [\#5305](https://github.com/ClickHouse/ClickHouse/issues/5305) [\#5801](https://github.com/ClickHouse/ClickHouse/pull/5801) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Invertir ngramSearch para que sea más intuitivo [\#5807](https://github.com/ClickHouse/ClickHouse/pull/5807) ([Más información](https://github.com/danlark1)) +- Agregar análisis de usuario en el generador de motor HDFS [\#5946](https://github.com/ClickHouse/ClickHouse/pull/5946) ([akonyaev90](https://github.com/akonyaev90)) +- Actualizar el valor predeterminado de `max_ast_elements parameter` [\#5933](https://github.com/ClickHouse/ClickHouse/pull/5933) ([Artem Konovalov](https://github.com/izebit)) +- Se agregó una noción de configuraciones obsoletas. La configuración obsoleta `allow_experimental_low_cardinality_type` se puede utilizar sin efecto. [0f15c01c6802f7ce1a1494c12c846be8c98944cd](https://github.com/ClickHouse/ClickHouse/commit/0f15c01c6802f7ce1a1494c12c846be8c98944cd) [Alexey Milovidov](https://github.com/alexey-milovidov) + +#### Mejora del rendimiento {#performance-improvement-4} + +- Aumente el número de secuencias a SELECT desde la tabla Merge para una distribución más uniforme de los subprocesos. Añadido `max_streams_multiplier_for_merge_tables`. Esto corrige [\#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [\#5915](https://github.com/ClickHouse/ClickHouse/pull/5915) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-7} + +- Agregue una prueba de compatibilidad con versiones anteriores para la interacción cliente-servidor con diferentes versiones de clickhouse. [\#5868](https://github.com/ClickHouse/ClickHouse/pull/5868) ([alesapin](https://github.com/alesapin)) +- Pruebe la información de cobertura en cada solicitud de confirmación y extracción. [\#5896](https://github.com/ClickHouse/ClickHouse/pull/5896) ([alesapin](https://github.com/alesapin)) +- Coopere con el desinfectante de direcciones para respaldar nuestros asignadores personalizados (`Arena` y `ArenaWithFreeLists`) para una mejor depuración de “use-after-free” error. [\#5728](https://github.com/ClickHouse/ClickHouse/pull/5728) ([akuzm](https://github.com/akuzm)) +- Cambiar a [Implementación de LLVM libunwind](https://github.com/llvm-mirror/libunwind) para el manejo de excepciones de C ++ y para la impresión de rastreos de pila [\#4828](https://github.com/ClickHouse/ClickHouse/pull/4828) ([Vídeos relacionados con nikita Lapkov](https://github.com/laplab)) +- Agregue dos advertencias más de -Weverything [\#5923](https://github.com/ClickHouse/ClickHouse/pull/5923) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir construir ClickHouse con Memory Sanitizer. [\#3949](https://github.com/ClickHouse/ClickHouse/pull/3949) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Informe ubsan fijo sobre `bitTest` función en la prueba de fuzz. [\#5943](https://github.com/ClickHouse/ClickHouse/pull/5943) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Docker: se agregó la posibilidad de iniciar una instancia de ClickHouse que requiere autenticación. [\#5727](https://github.com/ClickHouse/ClickHouse/pull/5727) ([Korviakov Andrey](https://github.com/shurshun)) +- Actualizar librdkafka a la versión 1.1.0 [\#5872](https://github.com/ClickHouse/ClickHouse/pull/5872) ([Ivan](https://github.com/abyss7)) +- Agregue el tiempo de espera global para las pruebas de integración y deshabilite algunas de ellas en el código de pruebas. [\#5741](https://github.com/ClickHouse/ClickHouse/pull/5741) ([alesapin](https://github.com/alesapin)) +- Solucionar algunas fallas ThreadSanitizer. [\#5854](https://github.com/ClickHouse/ClickHouse/pull/5854) ([akuzm](https://github.com/akuzm)) +- El `--no-undefined` opción obliga al enlazador a verificar la existencia de todos los nombres externos mientras se vincula. Es muy útil rastrear dependencias reales entre bibliotecas en el modo de compilación dividida. [\#5855](https://github.com/ClickHouse/ClickHouse/pull/5855) ([Ivan](https://github.com/abyss7)) +- Prueba de rendimiento añadida para [\#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [\#5914](https://github.com/ClickHouse/ClickHouse/pull/5914) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Compatibilidad fija con gcc-7. [\#5840](https://github.com/ClickHouse/ClickHouse/pull/5840) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó soporte para gcc-9. Esto corrige [\#5717](https://github.com/ClickHouse/ClickHouse/issues/5717) [\#5774](https://github.com/ClickHouse/ClickHouse/pull/5774) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el error cuando libunwind se puede vincular incorrectamente. [\#5948](https://github.com/ClickHouse/ClickHouse/pull/5948) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigieron algunas advertencias encontradas por PVS-Studio. [\#5921](https://github.com/ClickHouse/ClickHouse/pull/5921) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó soporte inicial para `clang-tidy` analizador estático. [\#5806](https://github.com/ClickHouse/ClickHouse/pull/5806) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Convertir macros endian BSD/Linux( ‘be64toh’ y ‘htobe64’) a los equivalentes de Mac OS X [\#5785](https://github.com/ClickHouse/ClickHouse/pull/5785) ([Fuente Chen](https://github.com/fredchenbj)) +- Guía de pruebas de integración mejorada. [\#5796](https://github.com/ClickHouse/ClickHouse/pull/5796) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Reparación de compilación en macosx + gcc9 [\#5822](https://github.com/ClickHouse/ClickHouse/pull/5822) ([filimonov](https://github.com/filimonov)) +- Corrige un error tipográfico difícil de detectar: aggreAGte -\> agregado. [\#5753](https://github.com/ClickHouse/ClickHouse/pull/5753) ([akuzm](https://github.com/akuzm)) +- Arreglar la compilación freebsd [\#5760](https://github.com/ClickHouse/ClickHouse/pull/5760) ([propulsor](https://github.com/proller)) +- Añadir enlace al canal experimental de YouTube al sitio web [\#5845](https://github.com/ClickHouse/ClickHouse/pull/5845) ([Ivan Blinkov](https://github.com/blinkov)) +- CMake: agregar opción para indicadores de cobertura: WITH\_COVERAGE [\#5776](https://github.com/ClickHouse/ClickHouse/pull/5776) ([propulsor](https://github.com/proller)) +- Corrige el tamaño inicial de algunos PODArray en línea. [\#5787](https://github.com/ClickHouse/ClickHouse/pull/5787) ([akuzm](https://github.com/akuzm)) +- clickhouse-servidor.postinst: arreglar la detección del sistema operativo para centos 6 [\#5788](https://github.com/ClickHouse/ClickHouse/pull/5788) ([propulsor](https://github.com/proller)) +- Se agregó la generación de paquetes Arch Linux. [\#5719](https://github.com/ClickHouse/ClickHouse/pull/5719) ([Vladimir Chebotarev](https://github.com/excitoon)) +- División Común / config.h por libs (dbms) [\#5715](https://github.com/ClickHouse/ClickHouse/pull/5715) ([propulsor](https://github.com/proller)) +- Correcciones para “Arcadia” plataforma de construcción [\#5795](https://github.com/ClickHouse/ClickHouse/pull/5795) ([propulsor](https://github.com/proller)) +- Correcciones para construcción no convencional (gcc9, sin submódulos) [\#5792](https://github.com/ClickHouse/ClickHouse/pull/5792) ([propulsor](https://github.com/proller)) +- Requerir un tipo explícito en unignmentStore porque se demostró que era propenso a errores [\#5791](https://github.com/ClickHouse/ClickHouse/pull/5791) ([akuzm](https://github.com/akuzm)) +- Corrige la compilación de MacOS [\#5830](https://github.com/ClickHouse/ClickHouse/pull/5830) ([filimonov](https://github.com/filimonov)) +- Prueba de rendimiento relativa a la nueva característica JIT con un conjunto de datos más grande, como se solicita aquí [\#5263](https://github.com/ClickHouse/ClickHouse/issues/5263) [\#5887](https://github.com/ClickHouse/ClickHouse/pull/5887) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Ejecutar pruebas con estado en la prueba de esfuerzo [12693e568722f11e19859742f56428455501fd2a](https://github.com/ClickHouse/ClickHouse/commit/12693e568722f11e19859742f56428455501fd2a) ([alesapin](https://github.com/alesapin)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-7} + +- `Kafka` está roto en esta versión. +- Permitir `adaptive_index_granularity` = 10MB por defecto para nuevo `MergeTree` tabla. Si creó nuevas tablas MergeTree en la versión 19.11+, será imposible degradar a versiones anteriores a la 19.6. [\#5628](https://github.com/ClickHouse/ClickHouse/pull/5628) ([alesapin](https://github.com/alesapin)) +- Eliminado diccionarios incrustados no documentados obsoletos que fueron utilizados por Yandex.Métrica. Función `OSIn`, `SEIn`, `OSToRoot`, `SEToRoot`, `OSHierarchy`, `SEHierarchy` ya no están disponibles. Si está utilizando estas funciones, escriba un correo electrónico a clickhouse-feedback@yandex-team.com. Nota: en el último momento decidimos mantener estas funciones por un tiempo. [\#5780](https://github.com/ClickHouse/ClickHouse/pull/5780) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +## Lanzamiento de ClickHouse 19.10 {#clickhouse-release-19-10} + +### Lanzamiento de ClickHouse 19.10.1.5, 2019-07-12 {#clickhouse-release-19-10-1-5-2019-07-12} + +#### Novedad {#new-feature-7} + +- Agregar nuevo códec de columna: `T64`. Hecho para columnas (U) IntX / EnumX / Data (Time) / DecimalX. Debería ser bueno para columnas con valores de rango constante o pequeño. Codec en sí permite ampliar o reducir el tipo de datos sin volver a comprimir. [\#5557](https://github.com/ClickHouse/ClickHouse/pull/5557) ([Artem Zuikov](https://github.com/4ertus2)) +- Agregar motor de base de datos `MySQL` que permiten ver todas las tablas en el servidor MySQL remoto [\#5599](https://github.com/ClickHouse/ClickHouse/pull/5599) ([Invierno Zhang](https://github.com/zhang2014)) +- `bitmapContains` aplicación. Es 2 veces más rápido que `bitmapHasAny` si el segundo mapa de bits contiene un elemento. [\#5535](https://github.com/ClickHouse/ClickHouse/pull/5535) ([Zhichang Yu](https://github.com/yuzhichang)) +- Soporte para `crc32` función (con comportamiento exactamente como en MySQL o PHP). No lo use si necesita una función hash. [\#5661](https://github.com/ClickHouse/ClickHouse/pull/5661) ([Remen Ivan](https://github.com/BHYCHIK)) +- Aplicado `SYSTEM START/STOP DISTRIBUTED SENDS` consultas para controlar inserciones asincrónicas en `Distributed` tabla. [\#4935](https://github.com/ClickHouse/ClickHouse/pull/4935) ([Invierno Zhang](https://github.com/zhang2014)) + +#### Corrección de errores {#bug-fix-22} + +- Ignore los límites de ejecución de consultas y el tamaño máximo de partes para los límites de combinación mientras ejecuta mutaciones. [\#5659](https://github.com/ClickHouse/ClickHouse/pull/5659) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir un error que puede conducir a la desduplicación de bloques normales (extremadamente raros) y la inserción de bloques duplicados (más a menudo). [\#5549](https://github.com/ClickHouse/ClickHouse/pull/5549) ([alesapin](https://github.com/alesapin)) +- Fijación de la función `arrayEnumerateUniqRanked` para argumentos con matrices vacías [\#5559](https://github.com/ClickHouse/ClickHouse/pull/5559) ([propulsor](https://github.com/proller)) +- No se suscriba a los temas de Kafka sin la intención de sondear ningún mensaje. [\#5698](https://github.com/ClickHouse/ClickHouse/pull/5698) ([Ivan](https://github.com/abyss7)) +- Hacer configuración `join_use_nulls` no obtienen ningún efecto para los tipos que no pueden estar dentro de Nullable [\#5700](https://github.com/ClickHouse/ClickHouse/pull/5700) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fijo `Incorrect size of index granularity` error [\#5720](https://github.com/ClickHouse/ClickHouse/pull/5720) ([Coraxster](https://github.com/coraxster)) +- Repara el desbordamiento de conversión de flotador a decimal [\#5607](https://github.com/ClickHouse/ClickHouse/pull/5607) ([Coraxster](https://github.com/coraxster)) +- Búfer de descarga cuando `WriteBufferFromHDFS`se llama destructor. Esto corrige la escritura en `HDFS`. [\#5684](https://github.com/ClickHouse/ClickHouse/pull/5684) ([Xindong Peng](https://github.com/eejoin)) + +#### Mejora {#improvement-7} + +- Trate las celdas vacías en `CSV` como valores predeterminados cuando la configuración `input_format_defaults_for_omitted_fields` está habilitado. [\#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) +- Carga sin bloqueo de diccionarios externos. [\#5567](https://github.com/ClickHouse/ClickHouse/pull/5567) ([Vitaly Baranov](https://github.com/vitlibar)) +- Los tiempos de espera de red se pueden cambiar dinámicamente para las conexiones ya establecidas de acuerdo con la configuración. [\#4558](https://github.com/ClickHouse/ClickHouse/pull/4558) ([Konstantin Podshumok](https://github.com/podshumok)) +- Utilizar “public\_suffix\_list” para funciones `firstSignificantSubdomain`, `cutToFirstSignificantSubdomain`. Está usando una tabla hash perfecta generada por `gperf` con una lista generada a partir del archivo: https://publicsuffix.org/list/public\_suffix\_list.dat. (por ejemplo, ahora reconocemos el dominio `ac.uk` como no significativo). [\#5030](https://github.com/ClickHouse/ClickHouse/pull/5030) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Adoptar `IPv6` tipo de datos en las tablas del sistema; columnas de información de cliente unificadas en `system.processes` y `system.query_log` [\#5640](https://github.com/ClickHouse/ClickHouse/pull/5640) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Uso de sesiones para conexiones con el protocolo de compatibilidad MySQL. Acerca de nosotros [\#5646](https://github.com/ClickHouse/ClickHouse/pull/5646) ([Yuriy Baranov](https://github.com/yurriy)) +- Apoyo más `ALTER` consulta `ON CLUSTER`. [\#5593](https://github.com/ClickHouse/ClickHouse/pull/5593) [\#5613](https://github.com/ClickHouse/ClickHouse/pull/5613) ([Sundyli](https://github.com/sundy-li)) +- Apoyo `` sección en `clickhouse-local` archivo de configuración. [\#5540](https://github.com/ClickHouse/ClickHouse/pull/5540) ([propulsor](https://github.com/proller)) +- Permitir ejecutar consulta con `remote` función de la tabla en `clickhouse-local` [\#5627](https://github.com/ClickHouse/ClickHouse/pull/5627) ([propulsor](https://github.com/proller)) + +#### Mejora del rendimiento {#performance-improvement-5} + +- Agregue la posibilidad de escribir la marca final al final de las columnas MergeTree. Permite evitar lecturas inútiles para las claves que están fuera del rango de datos de la tabla. Está habilitado sólo si se está utilizando granularidad de índice adaptativo. [\#5624](https://github.com/ClickHouse/ClickHouse/pull/5624) ([alesapin](https://github.com/alesapin)) +- Mejora del rendimiento de las tablas MergeTree en sistemas de archivos muy lentos al reducir el número de `stat` syscalls. [\#5648](https://github.com/ClickHouse/ClickHouse/pull/5648) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la degradación del rendimiento en la lectura de las tablas MergeTree que se introdujo en la versión 19.6. Correcciones \#5631. [\#5633](https://github.com/ClickHouse/ClickHouse/pull/5633) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-8} + +- Aplicado `TestKeeper` como una implementación de la interfaz ZooKeeper utilizada para probar [\#5643](https://github.com/ClickHouse/ClickHouse/pull/5643) ([alexey-milovidov](https://github.com/alexey-milovidov)) ([levushkin aleksej](https://github.com/alexey-milovidov)) +- A partir de ahora `.sql` las pruebas se pueden ejecutar aisladas por el servidor, en paralelo, con una base de datos aleatoria. Permite ejecutarlos más rápido, agregar nuevas pruebas con configuraciones de servidor personalizadas y asegurarse de que las diferentes pruebas no se afecten entre sí. [\#5554](https://github.com/ClickHouse/ClickHouse/pull/5554) ([Ivan](https://github.com/abyss7)) +- Quitar `` y `` de pruebas de rendimiento [\#5672](https://github.com/ClickHouse/ClickHouse/pull/5672) ([Olga Khvostikova](https://github.com/stavrolia)) +- Fijo “select\_format” prueba de rendimiento para `Pretty` formato [\#5642](https://github.com/ClickHouse/ClickHouse/pull/5642) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +## Lanzamiento de ClickHouse 19.9 {#clickhouse-release-19-9} + +### Lanzamiento de ClickHouse 19.9.3.31, 2019-07-05 {#clickhouse-release-19-9-3-31-2019-07-05} + +#### Corrección de errores {#bug-fix-23} + +- Corregir segfault en el códec Delta que afecta a las columnas con valores de menos de 32 bits de tamaño. El error llevó a la corrupción de memoria aleatoria. [\#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) +- Corregir un error raro en la comprobación de la parte con la columna LowCardinality. [\#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) +- Fix segfault en TTL se fusionan con columnas no físicas en el bloque. [\#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) +- Repara el potencial de sueño infinito de consultas de baja prioridad. [\#5842](https://github.com/ClickHouse/ClickHouse/pull/5842) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corrige cómo ClickHouse determina la zona horaria predeterminada como UCT en lugar de UTC. [\#5828](https://github.com/ClickHouse/ClickHouse/pull/5828) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir un error sobre la ejecución de las consultas distribuidas DROP / ALTER / TRUNCATE / OPTIMIZE ON CLUSTER en la réplica de seguidor antes de la réplica de líder. Ahora se ejecutarán directamente en réplica líder. [\#5757](https://github.com/ClickHouse/ClickHouse/pull/5757) ([alesapin](https://github.com/alesapin)) +- Corregir la condición de carrera, lo que hace que algunas consultas no aparezcan en query\_log instantáneamente después de la consulta SYSTEM FLUSH LOGS. [\#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) +- Se agregó soporte faltante para argumentos constantes para `evalMLModel` función. [\#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.9.2.4, 2019-06-24 {#clickhouse-release-19-9-2-4-2019-06-24} + +#### Novedad {#new-feature-8} + +- Imprima información sobre piezas congeladas en `system.parts` tabla. [\#5471](https://github.com/ClickHouse/ClickHouse/pull/5471) ([propulsor](https://github.com/proller)) +- Pregunte la contraseña del cliente en clickhouse-client start en tty si no se establece en argumentos [\#5092](https://github.com/ClickHouse/ClickHouse/pull/5092) ([propulsor](https://github.com/proller)) +- Implementar `dictGet` y `dictGetOrDefault` funciones para los tipos Decimal. [\#5394](https://github.com/ClickHouse/ClickHouse/pull/5394) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejora {#improvement-8} + +- Debian init: Agregar tiempo de espera de parada de servicio [\#5522](https://github.com/ClickHouse/ClickHouse/pull/5522) ([propulsor](https://github.com/proller)) +- Agregar configuración prohibida de forma predeterminada para crear tabla con tipos sospechosos para LowCardinality [\#5448](https://github.com/ClickHouse/ClickHouse/pull/5448) ([Olga Khvostikova](https://github.com/stavrolia)) +- Las funciones de regresión devuelven pesos del modelo cuando no se usan como Estado en función `evalMLMethod`. [\#5411](https://github.com/ClickHouse/ClickHouse/pull/5411) ([Quid37](https://github.com/Quid37)) +- Cambiar el nombre y mejorar los métodos de regresión. [\#5492](https://github.com/ClickHouse/ClickHouse/pull/5492) ([Quid37](https://github.com/Quid37)) +- Interfaces más claras de los buscadores de cadenas. [\#5586](https://github.com/ClickHouse/ClickHouse/pull/5586) ([Más información](https://github.com/danlark1)) + +#### Corrección de errores {#bug-fix-24} + +- Corregir la posible pérdida de datos en Kafka [\#5445](https://github.com/ClickHouse/ClickHouse/pull/5445) ([Ivan](https://github.com/abyss7)) +- Repara el potencial bucle infinito en `PrettySpace` formato cuando se llama con cero columnas [\#5560](https://github.com/ClickHouse/ClickHouse/pull/5560) ([Olga Khvostikova](https://github.com/stavrolia)) +- Se corrigió el error de desbordamiento UInt32 en modelos lineales. Permitir modelo ML eval para argumento de modelo no const. [\#5516](https://github.com/ClickHouse/ClickHouse/pull/5516) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- `ALTER TABLE ... DROP INDEX IF EXISTS ...` no debe generar una excepción si el índice proporcionado no existe [\#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Reparar segfault con `bitmapHasAny` en subconsulta escalar [\#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) +- Se corrigió el error cuando el grupo de conexiones de replicación no vuelve a intentar resolver el host, incluso cuando se eliminó la caché DNS. [\#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) +- Fijo `ALTER ... MODIFY TTL` en ReplicatedMergeTree. [\#5539](https://github.com/ClickHouse/ClickHouse/pull/5539) ([Anton Popov](https://github.com/CurtizJ)) +- Repare INSERT en la tabla distribuida con la columna MATERIALIZED [\#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) +- Corregir la asignación incorrecta al truncar el almacenamiento de unión [\#5437](https://github.com/ClickHouse/ClickHouse/pull/5437) ([Método de codificación de datos:](https://github.com/TCeason)) +- En versiones recientes del paquete tzdata algunos de los archivos son enlaces simbólicos ahora. El mecanismo actual para detectar la zona horaria predeterminada se rompe y da nombres incorrectos para algunas zonas horarias. Ahora, al menos, forzamos el nombre de la zona horaria al contenido de TZ si se proporciona. [\#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) +- Solucione algunos casos extremadamente raros con el buscador MultiVolnitsky cuando las agujas constantes en suma tienen al menos 16 KB de largo. El algoritmo omitió o sobrescribió los resultados anteriores que pueden conducir al resultado incorrecto de `multiSearchAny`. [\#5588](https://github.com/ClickHouse/ClickHouse/pull/5588) ([Más información](https://github.com/danlark1)) +- Solucione el problema cuando la configuración de las solicitudes de ExternalData no podía usar la configuración de ClickHouse. Además, por ahora, configuración `date_time_input_format` y `low_cardinality_allow_in_native_format` no se puede usar debido a la ambigüedad de los nombres (en datos externos puede interpretarse como formato de tabla y en la consulta puede ser una configuración). [\#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Más información](https://github.com/danlark1)) +- Corregir un error cuando las piezas se eliminaron solo de FS sin dejarlas caer de Zookeeper. [\#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) +- Eliminar el registro de depuración del protocolo MySQL [\#5478](https://github.com/ClickHouse/ClickHouse/pull/5478) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Omitir ZNONODE durante el procesamiento de consultas DDL [\#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) +- Fijar la mezcla `UNION ALL` tipo de columna de resultado. Hubo casos con datos inconsistentes y tipos de columnas de columnas resultantes. [\#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) +- Lanzar una excepción en enteros incorrectos en `dictGetT` funciones en lugar de accidente. [\#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) +- Arregle element\_count y load\_factor incorrectos para el diccionario hash en `system.dictionaries` tabla. [\#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-9} + +- Construcción fija sin `Brotli` Soporte de compresión HTTP (`ENABLE_BROTLI=OFF` de la variable cmake). [\#5521](https://github.com/ClickHouse/ClickHouse/pull/5521) ([Anton Yuzhaninov](https://github.com/citrin)) +- Incluye rugido.h como rugiendo / rugiendo.h [\#5523](https://github.com/ClickHouse/ClickHouse/pull/5523) ([Películas De Sexo](https://github.com/orivej)) +- Corregir las advertencias de gcc9 en hyperscan (\# ¡la directiva de línea es malvada!) [\#5546](https://github.com/ClickHouse/ClickHouse/pull/5546) ([Más información](https://github.com/danlark1)) +- Corrige todas las advertencias al compilar con gcc-9. Soluciona algunos problemas de contribución. Repara gcc9 ICE y envíalo a bugzilla. [\#5498](https://github.com/ClickHouse/ClickHouse/pull/5498) ([Más información](https://github.com/danlark1)) +- Enlace fijo con lld [\#5477](https://github.com/ClickHouse/ClickHouse/pull/5477) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Eliminar especializaciones no utilizadas en diccionarios [\#5452](https://github.com/ClickHouse/ClickHouse/pull/5452) ([Artem Zuikov](https://github.com/4ertus2)) +- Pruebas de rendimiento de mejora para formatear y analizar tablas para diferentes tipos de archivos [\#5497](https://github.com/ClickHouse/ClickHouse/pull/5497) ([Olga Khvostikova](https://github.com/stavrolia)) +- Correcciones para la ejecución de pruebas en paralelo [\#5506](https://github.com/ClickHouse/ClickHouse/pull/5506) ([propulsor](https://github.com/proller)) +- Docker: use configuraciones de clickhouse-test [\#5531](https://github.com/ClickHouse/ClickHouse/pull/5531) ([propulsor](https://github.com/proller)) +- Solucionar compilación para FreeBSD [\#5447](https://github.com/ClickHouse/ClickHouse/pull/5447) ([propulsor](https://github.com/proller)) +- Impulso de actualización a 1.70 [\#5570](https://github.com/ClickHouse/ClickHouse/pull/5570) ([propulsor](https://github.com/proller)) +- Repara el clickhouse de compilación como submódulo [\#5574](https://github.com/ClickHouse/ClickHouse/pull/5574) ([propulsor](https://github.com/proller)) +- Mejorar las pruebas de rendimiento JSONExtract [\#5444](https://github.com/ClickHouse/ClickHouse/pull/5444) ([Vitaly Baranov](https://github.com/vitlibar)) + +## Lanzamiento de ClickHouse 19.8 {#clickhouse-release-19-8} + +### Lanzamiento de ClickHouse 19.8.3.8, 2019-06-11 {#clickhouse-release-19-8-3-8-2019-06-11} + +#### Novedad {#new-features} + +- Funciones añadidas para trabajar con JSON [\#4686](https://github.com/ClickHouse/ClickHouse/pull/4686) ([Hcz](https://github.com/hczhcz)) [\#5124](https://github.com/ClickHouse/ClickHouse/pull/5124). ([Vitaly Baranov](https://github.com/vitlibar)) +- Agregue una función basename, con un comportamiento similar a una función basename, que existe en muchos idiomas (`os.path.basename` en python, `basename` in PHP, etc…). Work with both an UNIX-like path or a Windows path. [\#5136](https://github.com/ClickHouse/ClickHouse/pull/5136) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Añadir `LIMIT n, m BY` o `LIMIT m OFFSET n BY` sintaxis para establecer el desplazamiento de n para la cláusula LIMIT BY. [\#5138](https://github.com/ClickHouse/ClickHouse/pull/5138) ([Anton Popov](https://github.com/CurtizJ)) +- Añadido nuevo tipo de datos `SimpleAggregateFunction`, lo que permite tener columnas con agregación de luz en un `AggregatingMergeTree`. Esto solo se puede usar con funciones simples como `any`, `anyLast`, `sum`, `min`, `max`. [\#4629](https://github.com/ClickHouse/ClickHouse/pull/4629) ([Boris Granveaud](https://github.com/bgranvea)) +- Se agregó soporte para argumentos no constantes en función `ngramDistance` [\#5198](https://github.com/ClickHouse/ClickHouse/pull/5198) ([Más información](https://github.com/danlark1)) +- Funciones añadidas `skewPop`, `skewSamp`, `kurtPop` y `kurtSamp` para calcular la asimetría de la secuencia, la asimetría de la muestra, la curtosis y la curtosis de la muestra, respectivamente. [\#5200](https://github.com/ClickHouse/ClickHouse/pull/5200) ([Hcz](https://github.com/hczhcz)) +- La ayuda cambia de nombre la operación para `MaterializeView` almacenamiento. [\#5209](https://github.com/ClickHouse/ClickHouse/pull/5209) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Se agregó un servidor que permite conectarse a ClickHouse usando el cliente MySQL. [\#4715](https://github.com/ClickHouse/ClickHouse/pull/4715) ([Yuriy Baranov](https://github.com/yurriy)) +- Añadir `toDecimal*OrZero` y `toDecimal*OrNull` función. [\#5291](https://github.com/ClickHouse/ClickHouse/pull/5291) ([Artem Zuikov](https://github.com/4ertus2)) +- Soporte tipos decimales en funciones: `quantile`, `quantiles`, `median`, `quantileExactWeighted`, `quantilesExactWeighted`, medianExactWeighted. [\#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `toValidUTF8` function, which replaces all invalid UTF-8 characters by replacement character � (U+FFFD). [\#5322](https://github.com/ClickHouse/ClickHouse/pull/5322) ([Más información](https://github.com/danlark1)) +- Añadir `format` función. Formateo del patrón constante (patrón de formato Python simplificado) con las cadenas enumeradas en los argumentos. [\#5330](https://github.com/ClickHouse/ClickHouse/pull/5330) ([Más información](https://github.com/danlark1)) +- Añadir `system.detached_parts` que contiene información sobre las partes separadas de `MergeTree` tabla. [\#5353](https://github.com/ClickHouse/ClickHouse/pull/5353) ([akuzm](https://github.com/akuzm)) +- Añadir `ngramSearch` función para calcular la diferencia no simétrica entre la aguja y el pajar. [\#5418](https://github.com/ClickHouse/ClickHouse/pull/5418)[\#5422](https://github.com/ClickHouse/ClickHouse/pull/5422) ([Más información](https://github.com/danlark1)) +- Implementación de métodos básicos de aprendizaje automático (regresión lineal estocástica y regresión logística) utilizando la interfaz de funciones agregadas. Tiene diferentes estrategias para actualizar los pesos del modelo (descenso de gradiente simple, método de impulso, método Nesterov). También es compatible con mini lotes de tamaño personalizado. [\#4943](https://github.com/ClickHouse/ClickHouse/pull/4943) ([Quid37](https://github.com/Quid37)) +- Implementación de `geohashEncode` y `geohashDecode` función. [\#5003](https://github.com/ClickHouse/ClickHouse/pull/5003) ([Vasily Nemkov](https://github.com/Enmk)) +- Función agregada agregada `timeSeriesGroupSum`, que puede agregar diferentes series de tiempo que muestran la marca de tiempo no la alineación. Utilizará la interpolación lineal entre dos marcas de tiempo de muestra y luego sumará series temporales juntas. Función agregada agregada `timeSeriesGroupRateSum`, que calcula la tasa de series temporales y luego suma las tasas juntas. [\#4542](https://github.com/ClickHouse/ClickHouse/pull/4542) ([Información adicional](https://github.com/LiuYangkuan)) +- Funciones añadidas `IPv4CIDRtoIPv4Range` y `IPv6CIDRtoIPv6Range` para calcular los límites inferiores y superiores para una IP en la subred utilizando un CIDR. [\#5095](https://github.com/ClickHouse/ClickHouse/pull/5095) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Agregue un encabezado X-ClickHouse-Summary cuando enviamos una consulta usando HTTP con la configuración habilitada `send_progress_in_http_headers`. Devuelve la información habitual de X-ClickHouse-Progress, con información adicional como cuántas filas y bytes se insertaron en la consulta. [\#5116](https://github.com/ClickHouse/ClickHouse/pull/5116) ([Guillaume Tassery](https://github.com/YiuRULE)) + +#### Mejora {#improvements} + +- Añadir `max_parts_in_total` configuración para la familia de tablas MergeTree (predeterminado: 100 000) que evita la especificación insegura de la clave de partición \# 5166. [\#5171](https://github.com/ClickHouse/ClickHouse/pull/5171) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `clickhouse-obfuscator`: derivar la semilla para columnas individuales combinando la semilla inicial con el nombre de la columna, no la posición de la columna. Esto está destinado a transformar conjuntos de datos con varias tablas relacionadas, de modo que las tablas permanezcan JOINable después de la transformación. [\#5178](https://github.com/ClickHouse/ClickHouse/pull/5178) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Funciones añadidas `JSONExtractRaw`, `JSONExtractKeyAndValues`. Funciones renombradas `jsonExtract` a `JSONExtract`. Cuando algo sale mal, estas funciones devuelven los valores correspondientes, no `NULL`. Función modificada `JSONExtract`, ahora obtiene el tipo de devolución de su último parámetro y no inyecta nullables. Implementado de respaldo a RapidJSON en caso de que las instrucciones AVX2 no estén disponibles. Biblioteca Simdjson actualizado a una nueva versión. [\#5235](https://github.com/ClickHouse/ClickHouse/pull/5235) ([Vitaly Baranov](https://github.com/vitlibar)) +- Ahora `if` y `multiIf` funciones no dependen de la condición `Nullable`, pero confíe en las ramas para la compatibilidad sql. [\#5238](https://github.com/ClickHouse/ClickHouse/pull/5238) ([Jian Wu](https://github.com/janplus)) +- `In` predicado ahora genera `Null` resultado de `Null` entrada como el `Equal` función. [\#5152](https://github.com/ClickHouse/ClickHouse/pull/5152) ([Jian Wu](https://github.com/janplus)) +- Verifique el límite de tiempo cada (flush\_interval / poll\_timeout) número de filas de Kafka. Esto permite romper la lectura del consumidor de Kafka con más frecuencia y verificar los límites de tiempo para las transmisiones de nivel superior [\#5249](https://github.com/ClickHouse/ClickHouse/pull/5249) ([Ivan](https://github.com/abyss7)) +- Enlace rdkafka con SASL incluido. Debería permitir el uso de autenticación SASL SCRAM [\#5253](https://github.com/ClickHouse/ClickHouse/pull/5253) ([Ivan](https://github.com/abyss7)) +- Versión por lotes de RowRefList para TODAS LAS UNIONES. [\#5267](https://github.com/ClickHouse/ClickHouse/pull/5267) ([Artem Zuikov](https://github.com/4ertus2)) +- clickhouse-server: mensajes de error de escucha más informativos. [\#5268](https://github.com/ClickHouse/ClickHouse/pull/5268) ([propulsor](https://github.com/proller)) +- Diccionarios de soporte en clickhouse-copiadora para funciones en `` [\#5270](https://github.com/ClickHouse/ClickHouse/pull/5270) ([propulsor](https://github.com/proller)) +- Añadir nueva configuración `kafka_commit_every_batch` para regular Kafka cometer política. + Permite establecer el modo de confirmación: después de que se maneje cada lote de mensajes, o después de que se escriba todo el bloque en el almacenamiento. Es una compensación entre perder algunos mensajes o leerlos dos veces en algunas situaciones extremas. [\#5308](https://github.com/ClickHouse/ClickHouse/pull/5308) ([Ivan](https://github.com/abyss7)) +- Hacer `windowFunnel` soporta otros tipos de enteros sin firmar. [\#5320](https://github.com/ClickHouse/ClickHouse/pull/5320) ([Sundyli](https://github.com/sundy-li)) +- Permitir sombrear la columna virtual `_table` en el motor de fusión. [\#5325](https://github.com/ClickHouse/ClickHouse/pull/5325) ([Ivan](https://github.com/abyss7)) +- Hacer `sequenceMatch` Las funciones de agregado admiten otros tipos de enteros sin signo [\#5339](https://github.com/ClickHouse/ClickHouse/pull/5339) ([Sundyli](https://github.com/sundy-li)) +- Mejores mensajes de error si la suma de comprobación no coincide probablemente es causada por fallas de hardware. [\#5355](https://github.com/ClickHouse/ClickHouse/pull/5355) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Compruebe que las tablas subyacentes admiten `StorageMerge` [\#5366](https://github.com/ClickHouse/ClickHouse/pull/5366) ([Ivan](https://github.com/abyss7)) +- Сlose MySQL connections after their usage in external dictionaries. It is related to issue \#893. [\#5395](https://github.com/ClickHouse/ClickHouse/pull/5395) ([Clément Rodriguez](https://github.com/clemrodriguez)) +- Mejoras de MySQL Wire Protocol. Se ha cambiado el nombre del formato a MySQLWire. Usando RAII para llamar a RSA\_free. Deshabilitar SSL si no se puede crear contexto. [\#5419](https://github.com/ClickHouse/ClickHouse/pull/5419) ([Yuriy Baranov](https://github.com/yurriy)) +- clickhouse-client: allow to run with unaccessable history file (read-only, no disk space, file is directory, …). [\#5431](https://github.com/ClickHouse/ClickHouse/pull/5431) ([propulsor](https://github.com/proller)) +- Respete la configuración de consulta en INSERT asincrónicos en tablas distribuidas. [\#4936](https://github.com/ClickHouse/ClickHouse/pull/4936) ([Método de codificación de datos:](https://github.com/TCeason)) +- Funciones renombradas `leastSqr` a `simpleLinearRegression`, `LinearRegression` a `linearRegression`, `LogisticRegression` a `logisticRegression`. [\#5391](https://github.com/ClickHouse/ClickHouse/pull/5391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Mejoras de rendimiento {#performance-improvements} + +- Paralelice el procesamiento de partes de tablas MergeTree no replicadas en la consulta ALTER MODIFY. [\#4639](https://github.com/ClickHouse/ClickHouse/pull/4639) ([Ivan Kush](https://github.com/IvanKush)) +- Optimizaciones en la extracción de expresiones regulares. [\#5193](https://github.com/ClickHouse/ClickHouse/pull/5193) [\#5191](https://github.com/ClickHouse/ClickHouse/pull/5191) ([Más información](https://github.com/danlark1)) +- No agregue la columna de clave de unión correcta para unir el resultado si solo se usa en la sección join on . [\#5260](https://github.com/ClickHouse/ClickHouse/pull/5260) ([Artem Zuikov](https://github.com/4ertus2)) +- Congele el búfer Kafka después de la primera respuesta vacía. Evita múltiples invocaciones de `ReadBuffer::next()` para el resultado vacío en algunas secuencias de análisis de filas. [\#5283](https://github.com/ClickHouse/ClickHouse/pull/5283) ([Ivan](https://github.com/abyss7)) +- `concat` optimización de funciones para múltiples argumentos. [\#5357](https://github.com/ClickHouse/ClickHouse/pull/5357) ([Más información](https://github.com/danlark1)) +- Query optimisation. Allow push down IN statement while rewriting commа/cross join into inner one. [\#5396](https://github.com/ClickHouse/ClickHouse/pull/5396) ([Artem Zuikov](https://github.com/4ertus2)) +- Actualice nuestra implementación LZ4 con la referencia uno para tener una descompresión más rápida. [\#5070](https://github.com/ClickHouse/ClickHouse/pull/5070) ([Más información](https://github.com/danlark1)) +- Implementado MSD radix ordenar (basado en kxsort), y la clasificación parcial. [\#5129](https://github.com/ClickHouse/ClickHouse/pull/5129) ([Evgenii Pravda](https://github.com/kvinty)) + +#### Corrección de errores {#bug-fixes} + +- Fix push requiere columnas con unión [\#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Invierno Zhang](https://github.com/zhang2014)) +- Corregido error, cuando ClickHouse es ejecutado por systemd, el comando `sudo service clickhouse-server forcerestart` no funcionaba como se esperaba. [\#5204](https://github.com/ClickHouse/ClickHouse/pull/5204) ([propulsor](https://github.com/proller)) +- Corregir los códigos de error http en DataPartsExchange (el servidor http deinterserver en el puerto 9009 siempre devolvió el código 200, incluso en errores). [\#5216](https://github.com/ClickHouse/ClickHouse/pull/5216) ([propulsor](https://github.com/proller)) +- Repara SimpleAggregateFunction para String más largo que MAX\_SMALL\_STRING\_SIZE [\#5311](https://github.com/ClickHouse/ClickHouse/pull/5311) ([Azat Khuzhin](https://github.com/azat)) +- Corregir error para `Decimal` a `Nullable(Decimal)` conversión en IN. Admite otras conversiones de decimales a decimales (incluidas diferentes escalas). [\#5350](https://github.com/ClickHouse/ClickHouse/pull/5350) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el clobbering de FPU en la biblioteca simdjson que conducía a un cálculo incorrecto de `uniqHLL` y `uniqCombined` función agregada y funciones matemáticas tales como `log`. [\#5354](https://github.com/ClickHouse/ClickHouse/pull/5354) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el manejo de casos mixtos const / nonconst en funciones JSON. [\#5435](https://github.com/ClickHouse/ClickHouse/pull/5435) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fijar `retention` función. Ahora todas las condiciones que cumplen en una fila de datos se agregan al estado de datos. [\#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) +- Corregir el tipo de resultado para `quantileExact` con decimales. [\#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Documentación {#documentation} + +- Traducir documentación para `CollapsingMergeTree` a chino. [\#5168](https://github.com/ClickHouse/ClickHouse/pull/5168) ([张风啸](https://github.com/AlexZFX)) +- Traduzca alguna documentación sobre motores de mesa al chino. + [\#5134](https://github.com/ClickHouse/ClickHouse/pull/5134) + [\#5328](https://github.com/ClickHouse/ClickHouse/pull/5328) + ([nunca lee](https://github.com/neverlee)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements} + +- Corrija algunos informes de desinfectantes que muestran un uso probable después de liberar.[\#5139](https://github.com/ClickHouse/ClickHouse/pull/5139) [\#5143](https://github.com/ClickHouse/ClickHouse/pull/5143) [\#5393](https://github.com/ClickHouse/ClickHouse/pull/5393) ([Ivan](https://github.com/abyss7)) +- Mueva las pruebas de rendimiento fuera de directorios separados para mayor comodidad. [\#5158](https://github.com/ClickHouse/ClickHouse/pull/5158) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir pruebas de rendimiento incorrectas. [\#5255](https://github.com/ClickHouse/ClickHouse/pull/5255) ([alesapin](https://github.com/alesapin)) +- Se agregó una herramienta para calcular las sumas de comprobación causadas por los saltos de bits para depurar problemas de hardware. [\#5334](https://github.com/ClickHouse/ClickHouse/pull/5334) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Hacer script corredor más utilizable. [\#5340](https://github.com/ClickHouse/ClickHouse/pull/5340)[\#5360](https://github.com/ClickHouse/ClickHouse/pull/5360) ([filimonov](https://github.com/filimonov)) +- Agregue pequeñas instrucciones sobre cómo escribir pruebas de rendimiento. [\#5408](https://github.com/ClickHouse/ClickHouse/pull/5408) ([alesapin](https://github.com/alesapin)) +- Añadir capacidad para hacer sustituciones en crear, llenar y soltar consulta en las pruebas de rendimiento [\#5367](https://github.com/ClickHouse/ClickHouse/pull/5367) ([Olga Khvostikova](https://github.com/stavrolia)) + +## Lanzamiento de ClickHouse 19.7 {#clickhouse-release-19-7} + +### Lanzamiento de ClickHouse 19.7.5.29, 2019-07-05 {#clickhouse-release-19-7-5-29-2019-07-05} + +#### Corrección de errores {#bug-fix-25} + +- Corregir la regresión de rendimiento en algunas consultas con JOIN. [\#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Invierno Zhang](https://github.com/zhang2014)) + +### Lanzamiento de ClickHouse 19.7.5.27, 2019-06-09 {#clickhouse-release-19-7-5-27-2019-06-09} + +#### Novedad {#new-features-1} + +- Se agregaron funciones relacionadas con el mapa de bits `bitmapHasAny` y `bitmapHasAll` análogo a `hasAny` y `hasAll` funciones para matrices. [\#5279](https://github.com/ClickHouse/ClickHouse/pull/5279) ([Sergi Vladykin](https://github.com/svladykin)) + +#### Corrección de errores {#bug-fixes-1} + +- Arreglar segfault en `minmax` ÍNDICE con valor nulo. [\#5246](https://github.com/ClickHouse/ClickHouse/pull/5246) ([Nikita Vasilev](https://github.com/nikvas0)) +- Marque todas las columnas de entrada en LIMIT BY como salida requerida. Se fija ‘Not found column’ error en algunas consultas distribuidas. [\#5407](https://github.com/ClickHouse/ClickHouse/pull/5407) ([Constantin S. Pan](https://github.com/kvap)) +- Fijar “Column ‘0’ already exists” error en `SELECT .. PREWHERE` en la columna con DEFAULT [\#5397](https://github.com/ClickHouse/ClickHouse/pull/5397) ([propulsor](https://github.com/proller)) +- Fijar `ALTER MODIFY TTL` consulta sobre `ReplicatedMergeTree`. [\#5539](https://github.com/ClickHouse/ClickHouse/pull/5539/commits) ([Anton Popov](https://github.com/CurtizJ)) +- No bloquee el servidor cuando los consumidores de Kafka no se hayan iniciado. [\#5285](https://github.com/ClickHouse/ClickHouse/pull/5285) ([Ivan](https://github.com/abyss7)) +- Las funciones de mapa de bits fijas producen un resultado incorrecto. [\#5359](https://github.com/ClickHouse/ClickHouse/pull/5359) ([Cristina Andrés](https://github.com/andyyzh)) +- Fix element\_count para el diccionario hash (no incluye duplicados) [\#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) +- Utilice el contenido de la variable de entorno TZ como el nombre de la zona horaria. Ayuda a detectar correctamente la zona horaria predeterminada en algunos casos.[\#5443](https://github.com/ClickHouse/ClickHouse/pull/5443) ([Ivan](https://github.com/abyss7)) +- No intente convertir enteros en `dictGetT` funciones, porque no funciona correctamente. Lanza una excepción en su lugar. [\#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregir la configuración en la solicitud HTTP ExternalData. [\#5455](https://github.com/ClickHouse/ClickHouse/pull/5455) ([Danila + Kutenin](https://github.com/danlark1)) +- Corregir un error cuando las piezas se eliminaron solo de FS sin dejarlas caer de Zookeeper. [\#5520](https://github.com/ClickHouse/ClickHouse/pull/5520) ([alesapin](https://github.com/alesapin)) +- Fijar falla de segmentación en `bitmapHasAny` función. [\#5528](https://github.com/ClickHouse/ClickHouse/pull/5528) ([Zhichang Yu](https://github.com/yuzhichang)) +- Se corrigió el error cuando el grupo de conexiones de replicación no vuelve a intentar resolver el host, incluso cuando se eliminó la caché DNS. [\#5534](https://github.com/ClickHouse/ClickHouse/pull/5534) ([alesapin](https://github.com/alesapin)) +- Fijo `DROP INDEX IF EXISTS` consulta. Ahora `ALTER TABLE ... DROP INDEX IF EXISTS ...` query no genera una excepción si el índice proporcionado no existe. [\#5524](https://github.com/ClickHouse/ClickHouse/pull/5524) ([Gleb Novikov](https://github.com/NanoBjorn)) +- Corregir unión toda la columna de supertipo. Hubo casos con datos inconsistentes y tipos de columnas de columnas resultantes. [\#5503](https://github.com/ClickHouse/ClickHouse/pull/5503) ([Artem Zuikov](https://github.com/4ertus2)) +- Omita ZNONODE durante el procesamiento de consultas DDL. Antes, si otro nodo elimina el znode en la cola de tareas, el que + no lo procesó, pero ya obtuvo la lista de hijos, terminará el hilo DDLWorker. [\#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) +- Fije INSERT en la tabla Distributed () con la columna MATERIALIZED. [\#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) + +### Lanzamiento de ClickHouse 19.7.3.9, 2019-05-30 {#clickhouse-release-19-7-3-9-2019-05-30} + +#### Novedad {#new-features-2} + +- Permite limitar el rango de una configuración que puede especificar el usuario. + Estas restricciones se pueden configurar en el perfil de configuración del usuario. + [\#4931](https://github.com/ClickHouse/ClickHouse/pull/4931) ([Vitaly + Baranov](https://github.com/vitlibar)) +- Añadir una segunda versión de la función `groupUniqArray` con una opción + `max_size` parámetro que limita el tamaño de la matriz resultante. Este + comportamiento es similar a `groupArray(max_size)(x)` función. + [\#5026](https://github.com/ClickHouse/ClickHouse/pull/5026) ([Guillaume + Tassery](https://github.com/YiuRULE)) +- Para los formatos de archivo de entrada TSVWithNames/CSVWithNames, el orden de columnas ahora puede ser + determinado a partir del encabezado del archivo. Esto es controlado por + `input_format_with_names_use_header` parámetro. + [\#5081](https://github.com/ClickHouse/ClickHouse/pull/5081) + ([Alejandro](https://github.com/Akazz)) + +#### Corrección de errores {#bug-fixes-2} + +- Bloqueo con sin comprimir\_cache + JOIN durante la fusión (\# 5197) + [\#5133](https://github.com/ClickHouse/ClickHouse/pull/5133) ([Danila + Kutenin](https://github.com/danlark1)) +- Error de segmentación en una consulta de clickhouse-cliente a tablas del sistema. Acerca de nosotros + [\#5127](https://github.com/ClickHouse/ClickHouse/pull/5127) + ([Ivan](https://github.com/abyss7)) +- Pérdida de datos en carga pesada a través de KafkaEngine (\#4736) + [\#5080](https://github.com/ClickHouse/ClickHouse/pull/5080) + ([Ivan](https://github.com/abyss7)) +- Se corrigió una condición de carrera de datos muy rara que podría ocurrir al ejecutar una consulta con UNION ALL que involucraba al menos dos SELECT del sistema.columnas, sistema.mesas, sistema.partes, sistema.parts\_tables o tablas de la familia Merge y realizar ALTER de columnas de las tablas relacionadas simultáneamente. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejoras de rendimiento {#performance-improvements-1} + +- Utilice la clasificación de radix para ordenar por una sola columna numérica en `ORDER BY` sin + `LIMIT`. [\#5106](https://github.com/ClickHouse/ClickHouse/pull/5106), + [\#4439](https://github.com/ClickHouse/ClickHouse/pull/4439) + ([Evgenii Pravda](https://github.com/kvinty), + [alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Documentación {#documentation-1} + +- Traduzca la documentación de algunos motores de tabla al chino. + [\#5107](https://github.com/ClickHouse/ClickHouse/pull/5107), + [\#5094](https://github.com/ClickHouse/ClickHouse/pull/5094), + [\#5087](https://github.com/ClickHouse/ClickHouse/pull/5087) + ([张风啸](https://github.com/AlexZFX)), + [\#5068](https://github.com/ClickHouse/ClickHouse/pull/5068) ([nunca + Sotavento](https://github.com/neverlee)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-1} + +- Imprima los caracteres UTF-8 correctamente en `clickhouse-test`. + [\#5084](https://github.com/ClickHouse/ClickHouse/pull/5084) + ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregar parámetro de línea de comandos para clickhouse-cliente para cargar siempre la sugerencia + datos. [\#5102](https://github.com/ClickHouse/ClickHouse/pull/5102) + ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Resuelva algunas de las advertencias de PVS-Studio. + [\#5082](https://github.com/ClickHouse/ClickHouse/pull/5082) + ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar LZ4 [\#5040](https://github.com/ClickHouse/ClickHouse/pull/5040) ([Danila + Kutenin](https://github.com/danlark1)) +- Agregue gperf a los requisitos de compilación para la próxima solicitud de extracción \# 5030. + [\#5110](https://github.com/ClickHouse/ClickHouse/pull/5110) + ([propulsor](https://github.com/proller)) + +## Lanzamiento de ClickHouse 19.6 {#clickhouse-release-19-6} + +### Lanzamiento de ClickHouse 19.6.3.18, 2019-06-13 {#clickhouse-release-19-6-3-18-2019-06-13} + +#### Corrección de errores {#bug-fixes-3} + +- Fijo IN condición pushdown para consultas de funciones de tabla `mysql` y `odbc` y los correspondientes motores de mesa. Esto corrige \# 3540 y \# 2384. [\#5313](https://github.com/ClickHouse/ClickHouse/pull/5313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir el punto muerto en Zookeeper. [\#5297](https://github.com/ClickHouse/ClickHouse/pull/5297) ([github1youlc](https://github.com/github1youlc)) +- Permitir decimales citados en CSV. [\#5284](https://github.com/ClickHouse/ClickHouse/pull/5284) ([Artem Zuikov](https://github.com/4ertus2) +- No permitir la conversión de float Inf / NaN en Decimales (excepción de lanzamiento). [\#5282](https://github.com/ClickHouse/ClickHouse/pull/5282) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregir la carrera de datos en la consulta de cambio de nombre. [\#5247](https://github.com/ClickHouse/ClickHouse/pull/5247) ([Invierno Zhang](https://github.com/zhang2014)) +- Deshabilitar temporalmente LFAlloc. El uso de LFAlloc puede conducir a una gran cantidad de MAP\_FAILED en la asignación de UncompressedCache y, como resultado, a bloqueos de consultas en servidores de alta carga. [cfdba93](https://github.com/ClickHouse/ClickHouse/commit/cfdba938ce22f16efeec504f7f90206a515b1280)([Más información](https://github.com/danlark1)) + +### Lanzamiento de ClickHouse 19.6.2.11, 2019-05-13 {#clickhouse-release-19-6-2-11-2019-05-13} + +#### Novedad {#new-features-3} + +- Expresiones TTL para columnas y tablas. [\#4212](https://github.com/ClickHouse/ClickHouse/pull/4212) ([Anton Popov](https://github.com/CurtizJ)) +- Añadido soporte para `brotli` compresión para respuestas HTTP (Accept-Encoding: br) [\#4388](https://github.com/ClickHouse/ClickHouse/pull/4388) ([Mijaíl](https://github.com/fandyushin)) +- Se agregó una nueva función `isValidUTF8` para verificar si un conjunto de bytes está codificado correctamente en utf-8. [\#4934](https://github.com/ClickHouse/ClickHouse/pull/4934) ([Más información](https://github.com/danlark1)) +- Agregar nueva directiva de equilibrio de carga `first_or_random` que envía consultas al primer host especificado y, si es inaccesible, envía consultas a hosts aleatorios de fragmento. Útil para configuraciones de topología de replicación cruzada. [\#5012](https://github.com/ClickHouse/ClickHouse/pull/5012) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) + +#### Experimental Características {#experimental-features-1} + +- Añadir ajuste `index_granularity_bytes` (granularidad de índice adaptativo) para la familia de tablas MergeTree\*. [\#4826](https://github.com/ClickHouse/ClickHouse/pull/4826) ([alesapin](https://github.com/alesapin)) + +#### Mejora {#improvements-1} + +- Se agregó soporte para argumentos de tamaño y longitud no constantes y negativos para la función `substringUTF8`. [\#4989](https://github.com/ClickHouse/ClickHouse/pull/4989) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deshabilite el push-down a la tabla derecha en la unión izquierda, la tabla izquierda en la unión derecha y ambas tablas en unión completa. Esto corrige los resultados JOIN incorrectos en algunos casos. [\#4846](https://github.com/ClickHouse/ClickHouse/pull/4846) ([Ivan](https://github.com/abyss7)) +- `clickhouse-copier`: configuración de tareas de carga automática desde `--task-file` opcion [\#4876](https://github.com/ClickHouse/ClickHouse/pull/4876) ([propulsor](https://github.com/proller)) +- Se agregó el controlador de errores tipográficos para la fábrica de almacenamiento y la fábrica de funciones de tabla. [\#4891](https://github.com/ClickHouse/ClickHouse/pull/4891) ([Más información](https://github.com/danlark1)) +- Admite asteriscos y asteriscos calificados para combinaciones múltiples sin subconsultas [\#4898](https://github.com/ClickHouse/ClickHouse/pull/4898) ([Artem Zuikov](https://github.com/4ertus2)) +- Haga que el mensaje de error de columna faltante sea más fácil de usar. [\#4915](https://github.com/ClickHouse/ClickHouse/pull/4915) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejoras de rendimiento {#performance-improvements-2} + +- Aceleración significativa de ASOF JOIN [\#4924](https://github.com/ClickHouse/ClickHouse/pull/4924) ([Martijn Bakker](https://github.com/Gladdy)) + +#### Cambios incompatibles hacia atrás {#backward-incompatible-changes} + +- Encabezado HTTP `Query-Id` fue renombrado a `X-ClickHouse-Query-Id` para la consistencia. [\#4972](https://github.com/ClickHouse/ClickHouse/pull/4972) ([Mijaíl](https://github.com/fandyushin)) + +#### Corrección de errores {#bug-fixes-4} + +- Se corrigió la desreferencia del puntero nulo potencial en `clickhouse-copier`. [\#4900](https://github.com/ClickHouse/ClickHouse/pull/4900) ([propulsor](https://github.com/proller)) +- Se corrigió el error en la consulta con JOIN + ARRAY JOIN [\#4938](https://github.com/ClickHouse/ClickHouse/pull/4938) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el inicio del servidor cuando un diccionario depende de otro diccionario a través de una base de datos con motor = Diccionario. [\#4962](https://github.com/ClickHouse/ClickHouse/pull/4962) ([Vitaly Baranov](https://github.com/vitlibar)) +- Partially fix distributed\_product\_mode = local. It's possible to allow columns of local tables in where/having/order by/… via table aliases. Throw exception if table does not have alias. There's not possible to access to the columns without table aliases yet. [\#4986](https://github.com/ClickHouse/ClickHouse/pull/4986) ([Artem Zuikov](https://github.com/4ertus2)) +- Repara el resultado potencialmente incorrecto para `SELECT DISTINCT` con `JOIN` [\#5001](https://github.com/ClickHouse/ClickHouse/pull/5001) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió una condición de carrera de datos muy rara que podría ocurrir al ejecutar una consulta con UNION ALL que involucraba al menos dos SELECT del sistema.columnas, sistema.mesas, sistema.partes, sistema.parts\_tables o tablas de la familia Merge y realizar ALTER de columnas de las tablas relacionadas simultáneamente. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-2} + +- Se corrigieron errores de prueba al ejecutar clickhouse-server en un host diferente [\#4713](https://github.com/ClickHouse/ClickHouse/pull/4713) ([Vasily Nemkov](https://github.com/Enmk)) +- clickhouse-test: deshabilita las secuencias de control de color en un entorno no tty. [\#4937](https://github.com/ClickHouse/ClickHouse/pull/4937) ([alesapin](https://github.com/alesapin)) +- clickhouse-test: Permitir el uso de cualquier base de datos de prueba (eliminar `test.` calificación donde sea posible) [\#5008](https://github.com/ClickHouse/ClickHouse/pull/5008) ([propulsor](https://github.com/proller)) +- Corregir errores de ubsan [\#5037](https://github.com/ClickHouse/ClickHouse/pull/5037) ([Vitaly Baranov](https://github.com/vitlibar)) +- Yandex LFAlloc se agregó a ClickHouse para asignar datos de MarkCache y UncompressedCache de diferentes maneras para detectar segfaults más confiables [\#4995](https://github.com/ClickHouse/ClickHouse/pull/4995) ([Más información](https://github.com/danlark1)) +- Python utiliza para ayudar con backports y registros de cambios. [\#4949](https://github.com/ClickHouse/ClickHouse/pull/4949) ([Ivan](https://github.com/abyss7)) + +## Lanzamiento de ClickHouse 19.5 {#clickhouse-release-19-5} + +### Lanzamiento de ClickHouse 19.5.4.22, 2019-05-13 {#clickhouse-release-19-5-4-22-2019-05-13} + +#### Corrección de errores {#bug-fixes-5} + +- Se corrigió un posible bloqueo en las funciones de mapa de bits \* [\#5220](https://github.com/ClickHouse/ClickHouse/pull/5220) [\#5228](https://github.com/ClickHouse/ClickHouse/pull/5228) ([Cristina Andrés](https://github.com/andyyzh)) +- Se corrigió una condición de carrera de datos muy rara que podría ocurrir al ejecutar una consulta con UNION ALL que involucraba al menos dos SELECT del sistema.columnas, sistema.mesas, sistema.partes, sistema.parts\_tables o tablas de la familia Merge y realizar ALTER de columnas de las tablas relacionadas simultáneamente. [\#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Error fijo `Set for IN is not created yet in case of using single LowCardinality column in the left part of IN`. Este error ocurrió si la columna LowCardinality era la parte de la clave principal. \#5031 [\#5154](https://github.com/ClickHouse/ClickHouse/pull/5154) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Modificación de la función de retención: si una fila satisface la primera condición y la NTH, solo se agrega la primera condición satisfecha al estado de datos. Ahora todas las condiciones que cumplen en una fila de datos se agregan al estado de datos. [\#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) + +### Lanzamiento de ClickHouse 19.5.3.8, 2019-04-18 {#clickhouse-release-19-5-3-8-2019-04-18} + +#### Corrección de errores {#bug-fixes-6} + +- Tipo fijo de ajuste `max_partitions_per_insert_block` de booleano a UInt64. [\#5028](https://github.com/ClickHouse/ClickHouse/pull/5028) ([Más información](https://github.com/mhsekhavat)) + +### Lanzamiento de ClickHouse 19.5.2.6, 2019-04-15 {#clickhouse-release-19-5-2-6-2019-04-15} + +#### Novedad {#new-features-4} + +- [Hyperscan](https://github.com/intel/hyperscan) Se agregó múltiples coincidencias de expresiones regulares (funciones `multiMatchAny`, `multiMatchAnyIndex`, `multiFuzzyMatchAny`, `multiFuzzyMatchAnyIndex`). [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780), [\#4841](https://github.com/ClickHouse/ClickHouse/pull/4841) ([Más información](https://github.com/danlark1)) +- `multiSearchFirstPosition` se añadió la función. [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Más información](https://github.com/danlark1)) +- Implemente el filtro de expresión predefinido por fila para las tablas. [\#4792](https://github.com/ClickHouse/ClickHouse/pull/4792) ([Ivan](https://github.com/abyss7)) +- Un nuevo tipo de índices de omisión de datos basados en filtros de floración (se puede usar para `equal`, `in` y `like` función). [\#4499](https://github.com/ClickHouse/ClickHouse/pull/4499) ([Nikita Vasilev](https://github.com/nikvas0)) +- Añadir `ASOF JOIN` que permite ejecutar consultas que se unen al valor más reciente conocido. [\#4774](https://github.com/ClickHouse/ClickHouse/pull/4774) [\#4867](https://github.com/ClickHouse/ClickHouse/pull/4867) [\#4863](https://github.com/ClickHouse/ClickHouse/pull/4863) [\#4875](https://github.com/ClickHouse/ClickHouse/pull/4875) ([Martijn Bakker](https://github.com/Gladdy), [Artem Zuikov](https://github.com/4ertus2)) +- Reescribir múltiples `COMMA JOIN` a `CROSS JOIN`. Luego reescribirlos para `INNER JOIN` si es posible. [\#4661](https://github.com/ClickHouse/ClickHouse/pull/4661) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejora {#improvement-9} + +- `topK` y `topKWeighted` ahora soporta personalizado `loadFactor` (soluciona el problema [\#4252](https://github.com/ClickHouse/ClickHouse/issues/4252)). [\#4634](https://github.com/ClickHouse/ClickHouse/pull/4634) ([Kirill Danshin](https://github.com/kirillDanshin)) +- Permitir usar `parallel_replicas_count > 1` incluso para tablas sin muestreo (la configuración simplemente se ignora para ellas). En versiones anteriores fue conducido a la excepción. [\#4637](https://github.com/ClickHouse/ClickHouse/pull/4637) ([Alexey Elymanov](https://github.com/digitalist)) +- Soporte para `CREATE OR REPLACE VIEW`. Permite crear una vista o establecer una nueva definición en una sola instrucción. [\#4654](https://github.com/ClickHouse/ClickHouse/pull/4654) ([Boris Granveaud](https://github.com/bgranvea)) +- `Buffer` motor de mesa ahora soporta `PREWHERE`. [\#4671](https://github.com/ClickHouse/ClickHouse/pull/4671) ([Información adicional](https://github.com/LiuYangkuan)) +- Agregue la capacidad de iniciar una tabla replicada sin metadatos en zookeeper en `readonly` modo. [\#4691](https://github.com/ClickHouse/ClickHouse/pull/4691) ([alesapin](https://github.com/alesapin)) +- Parpadeo fijo de la barra de progreso en clickhouse-cliente. El problema fue más notable al usar `FORMAT Null` con consultas de streaming. [\#4811](https://github.com/ClickHouse/ClickHouse/pull/4811) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir deshabilitar funciones con `hyperscan` biblioteca por usuario para limitar el uso de recursos potencialmente excesivo e incontrolado. [\#4816](https://github.com/ClickHouse/ClickHouse/pull/4816) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue el registro del número de versión en todos los errores. [\#4824](https://github.com/ClickHouse/ClickHouse/pull/4824) ([propulsor](https://github.com/proller)) +- Se agregó restricción a la `multiMatch` funciones que requieren tamaño de cadena para encajar en `unsigned int`. También se agregó el número de límite de argumentos al `multiSearch` función. [\#4834](https://github.com/ClickHouse/ClickHouse/pull/4834) ([Más información](https://github.com/danlark1)) +- Uso mejorado del espacio de arañazos y manejo de errores en Hyperscan. [\#4866](https://github.com/ClickHouse/ClickHouse/pull/4866) ([Más información](https://github.com/danlark1)) +- Llenar `system.graphite_detentions` de una configuración de tabla de `*GraphiteMergeTree` mesas de motores. [\#4584](https://github.com/ClickHouse/ClickHouse/pull/4584) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Cambiar nombre `trigramDistance` función para `ngramDistance` y añadir más funciones con `CaseInsensitive` y `UTF`. [\#4602](https://github.com/ClickHouse/ClickHouse/pull/4602) ([Más información](https://github.com/danlark1)) +- Cálculo de índices de saltos de datos mejorado. [\#4640](https://github.com/ClickHouse/ClickHouse/pull/4640) ([Nikita Vasilev](https://github.com/nikvas0)) +- Mantener ordinario, `DEFAULT`, `MATERIALIZED` y `ALIAS` columnas en una sola lista (soluciona el problema [\#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [\#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Método de codificación de datos:](https://github.com/ztlpn)) + +#### Corrección de errores {#bug-fix-26} + +- Evitar `std::terminate` en caso de error de asignación de memoria. Ahora `std::bad_alloc` excepción se lanza como se esperaba. [\#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corrige la lectura de capnproto desde el búfer. A veces, los archivos no se cargaron correctamente por HTTP. [\#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) +- Corregir error `Unknown log entry type: 0` despues `OPTIMIZE TABLE FINAL` consulta. [\#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos pájaro](https://github.com/amosbird)) +- Argumentos erróneos para `hasAny` o `hasAll` funciones pueden conducir a segfault. [\#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Puede producirse un punto muerto durante la ejecución `DROP DATABASE dictionary` consulta. [\#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Repara el comportamiento indefinido en `median` y `quantile` función. [\#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([Hcz](https://github.com/hczhcz)) +- Corregir la detección de nivel de compresión cuando `network_compression_method` en minúsculas. Roto en v19.1. [\#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([propulsor](https://github.com/proller)) +- Se corrigió la ignorancia de `UTC` ajuste (soluciona el problema [\#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [\#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([propulsor](https://github.com/proller)) +- Fijar `histogram` comportamiento de la función con `Distributed` tabla. [\#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([Olegkv](https://github.com/olegkv)) +- Informe tsan fijo `destroy of a locked mutex`. [\#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el informe de TSan sobre el cierre debido a la condición de carrera en el uso de registros del sistema. Se corrigió el uso potencial después de liberar al apagar cuando part\_log está habilitado. [\#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar las piezas de nuevo control en `ReplicatedMergeTreeAlterThread` en caso de error. [\#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Las operaciones aritméticas en estados de función de agregado intermedios no funcionaban para argumentos constantes (como los resultados de subconsulta). [\#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Siempre retroceda los nombres de las columnas en los metadatos. De lo contrario, es imposible crear una tabla con una columna llamada `index` (el servidor no se reiniciará debido a `ATTACH` consulta en metadatos). [\#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix crash en `ALTER ... MODIFY ORDER BY` en `Distributed` tabla. [\#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([Método de codificación de datos:](https://github.com/TCeason)) +- Arreglar segfault en `JOIN ON` con habilitado `enable_optimize_predicate_expression`. [\#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Invierno Zhang](https://github.com/zhang2014)) +- Corregir un error al agregar una fila extraña después de consumir un mensaje protobuf de Kafka. [\#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregir el bloqueo de `JOIN` no aceptan vs columna que acepta valores null. Fijar `NULLs` en las teclas de la derecha en `ANY JOIN` + `join_use_nulls`. [\#4815](https://github.com/ClickHouse/ClickHouse/pull/4815) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar falla de segmentación en `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([propulsor](https://github.com/proller)) +- Condición de carrera fija en `SELECT` de `system.tables` si la tabla se cambia de nombre o se modifica simultáneamente. [\#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la carrera de datos al obtener una parte de datos que ya está obsoleta. [\#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la carrera de datos raros que puede ocurrir durante `RENAME` tabla de la familia MergeTree. [\#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fallo de segmentación fijo en la función `arrayIntersect`. La falla de segmentación podría ocurrir si se llamara a la función con argumentos constantes y ordinarios mixtos. [\#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) +- Lectura fija de `Array(LowCardinality)` columna en caso raro cuando la columna contenía una larga secuencia de matrices vacías. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fix crash en `FULL/RIGHT JOIN` cuando nos unimos en nullable vs no nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar `No message received` excepción al recuperar partes entre réplicas. [\#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) +- Fijo `arrayIntersect` resultado incorrecto de la función en caso de varios valores repetidos en una sola matriz. [\#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir una condición de carrera durante `ALTER COLUMN` consultas que podrían provocar un bloqueo del servidor (soluciona el problema [\#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [\#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Corregir el resultado incorrecto en `FULL/RIGHT JOIN` con la columna const. [\#4723](https://github.com/ClickHouse/ClickHouse/pull/4723) ([Artem Zuikov](https://github.com/4ertus2)) +- Reparar duplicados en `GLOBAL JOIN` con asterisk. [\#4705](https://github.com/ClickHouse/ClickHouse/pull/4705) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregir la deducción de parámetros en `ALTER MODIFY` de la columna `CODEC` cuando no se especifica el tipo de columna. [\#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) +- Función `cutQueryStringAndFragment()` y `queryStringAndFragment()` ahora funciona correctamente cuando `URL` contiene un fragmento y ninguna consulta. [\#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregir un error raro al configurar `min_bytes_to_use_direct_io` es mayor que cero, lo que ocurre cuando el hilo tiene que buscar hacia atrás en el archivo de columna. [\#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) +- Corregir tipos de argumentos incorrectos para funciones agregadas con `LowCardinality` argumentos (soluciona el problema [\#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [\#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir la calificación de nombre incorrecto en `GLOBAL JOIN`. [\#4969](https://github.com/ClickHouse/ClickHouse/pull/4969) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar la función `toISOWeek` resultado para el año 1970. [\#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `DROP`, `TRUNCATE` y `OPTIMIZE` duplicación de consultas, cuando se ejecuta en `ON CLUSTER` para `ReplicatedMergeTree*` tablas de la familia. [\#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) + +#### Cambio incompatible hacia atrás {#backward-incompatible-change-8} + +- Cambiar el nombre de la configuración `insert_sample_with_metadata` establecer `input_format_defaults_for_omitted_fields`. [\#4771](https://github.com/ClickHouse/ClickHouse/pull/4771) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadido `max_partitions_per_insert_block` (con valor 100 por defecto). Si el bloque insertado contiene un mayor número de particiones, se lanza una excepción. Establezca en 0 si desea eliminar el límite (no recomendado). [\#4845](https://github.com/ClickHouse/ClickHouse/pull/4845) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se renombraron las funciones de búsqueda múltiple (`multiPosition` a `multiSearchAllPositions`, `multiSearch` a `multiSearchAny`, `firstMatch` a `multiSearchFirstIndex`). [\#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Más información](https://github.com/danlark1)) + +#### Mejora del rendimiento {#performance-improvement-6} + +- Optimice el buscador Volnitsky al alinear, dando aproximadamente un 5-10% de mejora de búsqueda para consultas con muchas agujas o muchos bigrams similares. [\#4862](https://github.com/ClickHouse/ClickHouse/pull/4862) ([Más información](https://github.com/danlark1)) +- Solucionar problema de rendimiento al configurar `use_uncompressed_cache` es mayor que cero, que apareció cuando todos los datos leídos contenidos en la memoria caché. [\#4913](https://github.com/ClickHouse/ClickHouse/pull/4913) ([alesapin](https://github.com/alesapin)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-10} + +- Endurecimiento de la compilación de depuración: más asignaciones de memoria granulares y ASLR; agregue protección de memoria para caché de marcas e índice. Esto permite encontrar más errores de memoria en caso de que ASan y MSan no puedan hacerlo. [\#4632](https://github.com/ClickHouse/ClickHouse/pull/4632) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir soporte para variables cmake `ENABLE_PROTOBUF`, `ENABLE_PARQUET` y `ENABLE_BROTLI` que permite habilitar / deshabilitar las características anteriores (lo mismo que podemos hacer para librdkafka, mysql, etc.). [\#4669](https://github.com/ClickHouse/ClickHouse/pull/4669) ([Silviu Caragea](https://github.com/silviucpp)) +- Agregue la capacidad de imprimir la lista de procesos y los rastros de pila de todos los hilos si algunas consultas se cuelgan después de la ejecución de la prueba. [\#4675](https://github.com/ClickHouse/ClickHouse/pull/4675) ([alesapin](https://github.com/alesapin)) +- Agregar reintentos en `Connection loss` error en `clickhouse-test`. [\#4682](https://github.com/ClickHouse/ClickHouse/pull/4682) ([alesapin](https://github.com/alesapin)) +- Agregue freebsd build con vagrant y compile con desinfectante de hilos al script del empaquetador. [\#4712](https://github.com/ClickHouse/ClickHouse/pull/4712) [\#4748](https://github.com/ClickHouse/ClickHouse/pull/4748) ([alesapin](https://github.com/alesapin)) +- Ahora el usuario pidió contraseña para el usuario `'default'` durante la instalación. [\#4725](https://github.com/ClickHouse/ClickHouse/pull/4725) ([propulsor](https://github.com/proller)) +- Suprimir advertencia en `rdkafka` biblioteca. [\#4740](https://github.com/ClickHouse/ClickHouse/pull/4740) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir la capacidad de construir sin ssl. [\#4750](https://github.com/ClickHouse/ClickHouse/pull/4750) ([propulsor](https://github.com/proller)) +- Agregue una forma de iniciar una imagen de clickhouse-server desde un usuario personalizado. [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Actualice el impulso de contrib a 1.69. [\#4793](https://github.com/ClickHouse/ClickHouse/pull/4793) ([propulsor](https://github.com/proller)) +- Deshabilitar el uso de `mremap` cuando se compila con Thread Sanitizer. Sorprendentemente, TSan no intercepta `mremap` (aunque intercepta `mmap`, `munmap`) que conduce a falsos positivos. Informe TSan fijo en pruebas con estado. [\#4859](https://github.com/ClickHouse/ClickHouse/pull/4859) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue comprobación de prueba usando el esquema de formato a través de la interfaz HTTP. [\#4864](https://github.com/ClickHouse/ClickHouse/pull/4864) ([Vitaly Baranov](https://github.com/vitlibar)) + +## Lanzamiento de ClickHouse 19.4 {#clickhouse-release-19-4} + +### Lanzamiento de ClickHouse 19.4.4.33, 2019-04-17 {#clickhouse-release-19-4-4-33-2019-04-17} + +#### Corrección de errores {#bug-fixes-7} + +- Evitar `std::terminate` en caso de error de asignación de memoria. Ahora `std::bad_alloc` excepción se lanza como se esperaba. [\#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corrige la lectura de capnproto desde el búfer. A veces, los archivos no se cargaron correctamente por HTTP. [\#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) +- Corregir error `Unknown log entry type: 0` despues `OPTIMIZE TABLE FINAL` consulta. [\#4683](https://github.com/ClickHouse/ClickHouse/pull/4683) ([Amos pájaro](https://github.com/amosbird)) +- Argumentos erróneos para `hasAny` o `hasAll` funciones pueden conducir a segfault. [\#4698](https://github.com/ClickHouse/ClickHouse/pull/4698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Puede producirse un punto muerto durante la ejecución `DROP DATABASE dictionary` consulta. [\#4701](https://github.com/ClickHouse/ClickHouse/pull/4701) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Repara el comportamiento indefinido en `median` y `quantile` función. [\#4702](https://github.com/ClickHouse/ClickHouse/pull/4702) ([Hcz](https://github.com/hczhcz)) +- Corregir la detección de nivel de compresión cuando `network_compression_method` en minúsculas. Roto en v19.1. [\#4706](https://github.com/ClickHouse/ClickHouse/pull/4706) ([propulsor](https://github.com/proller)) +- Se corrigió la ignorancia de `UTC` ajuste (soluciona el problema [\#4658](https://github.com/ClickHouse/ClickHouse/issues/4658)). [\#4718](https://github.com/ClickHouse/ClickHouse/pull/4718) ([propulsor](https://github.com/proller)) +- Fijar `histogram` comportamiento de la función con `Distributed` tabla. [\#4741](https://github.com/ClickHouse/ClickHouse/pull/4741) ([Olegkv](https://github.com/olegkv)) +- Informe tsan fijo `destroy of a locked mutex`. [\#4742](https://github.com/ClickHouse/ClickHouse/pull/4742) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el informe de TSan sobre el cierre debido a la condición de carrera en el uso de registros del sistema. Se corrigió el uso potencial después de liberar al apagar cuando part\_log está habilitado. [\#4758](https://github.com/ClickHouse/ClickHouse/pull/4758) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar las piezas de nuevo control en `ReplicatedMergeTreeAlterThread` en caso de error. [\#4772](https://github.com/ClickHouse/ClickHouse/pull/4772) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Las operaciones aritméticas en estados de función de agregado intermedios no funcionaban para argumentos constantes (como los resultados de subconsulta). [\#4776](https://github.com/ClickHouse/ClickHouse/pull/4776) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Siempre retroceda los nombres de las columnas en los metadatos. De lo contrario, es imposible crear una tabla con una columna llamada `index` (el servidor no se reiniciará debido a `ATTACH` consulta en metadatos). [\#4782](https://github.com/ClickHouse/ClickHouse/pull/4782) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fix crash en `ALTER ... MODIFY ORDER BY` en `Distributed` tabla. [\#4790](https://github.com/ClickHouse/ClickHouse/pull/4790) ([Método de codificación de datos:](https://github.com/TCeason)) +- Arreglar segfault en `JOIN ON` con habilitado `enable_optimize_predicate_expression`. [\#4794](https://github.com/ClickHouse/ClickHouse/pull/4794) ([Invierno Zhang](https://github.com/zhang2014)) +- Corregir un error al agregar una fila extraña después de consumir un mensaje protobuf de Kafka. [\#4808](https://github.com/ClickHouse/ClickHouse/pull/4808) ([Vitaly Baranov](https://github.com/vitlibar)) +- Fijar falla de segmentación en `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([propulsor](https://github.com/proller)) +- Condición de carrera fija en `SELECT` de `system.tables` si la tabla se cambia de nombre o se modifica simultáneamente. [\#4836](https://github.com/ClickHouse/ClickHouse/pull/4836) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la carrera de datos al obtener una parte de datos que ya está obsoleta. [\#4839](https://github.com/ClickHouse/ClickHouse/pull/4839) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la carrera de datos raros que puede ocurrir durante `RENAME` tabla de la familia MergeTree. [\#4844](https://github.com/ClickHouse/ClickHouse/pull/4844) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fallo de segmentación fijo en la función `arrayIntersect`. La falla de segmentación podría ocurrir si se llamara a la función con argumentos constantes y ordinarios mixtos. [\#4847](https://github.com/ClickHouse/ClickHouse/pull/4847) ([Lixiang Qian](https://github.com/fancyqlx)) +- Lectura fija de `Array(LowCardinality)` columna en caso raro cuando la columna contenía una larga secuencia de matrices vacías. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fijar `No message received` excepción al recuperar partes entre réplicas. [\#4856](https://github.com/ClickHouse/ClickHouse/pull/4856) ([alesapin](https://github.com/alesapin)) +- Fijo `arrayIntersect` resultado incorrecto de la función en caso de varios valores repetidos en una sola matriz. [\#4871](https://github.com/ClickHouse/ClickHouse/pull/4871) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir una condición de carrera durante `ALTER COLUMN` consultas que podrían provocar un bloqueo del servidor (soluciona el problema [\#3421](https://github.com/ClickHouse/ClickHouse/issues/3421)). [\#4592](https://github.com/ClickHouse/ClickHouse/pull/4592) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Corregir la deducción de parámetros en `ALTER MODIFY` de la columna `CODEC` cuando no se especifica el tipo de columna. [\#4883](https://github.com/ClickHouse/ClickHouse/pull/4883) ([alesapin](https://github.com/alesapin)) +- Función `cutQueryStringAndFragment()` y `queryStringAndFragment()` ahora funciona correctamente cuando `URL` contiene un fragmento y ninguna consulta. [\#4894](https://github.com/ClickHouse/ClickHouse/pull/4894) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregir un error raro al configurar `min_bytes_to_use_direct_io` es mayor que cero, lo que ocurre cuando el hilo tiene que buscar hacia atrás en el archivo de columna. [\#4897](https://github.com/ClickHouse/ClickHouse/pull/4897) ([alesapin](https://github.com/alesapin)) +- Corregir tipos de argumentos incorrectos para funciones agregadas con `LowCardinality` argumentos (soluciona el problema [\#4919](https://github.com/ClickHouse/ClickHouse/issues/4919)). [\#4922](https://github.com/ClickHouse/ClickHouse/pull/4922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Fijar la función `toISOWeek` resultado para el año 1970. [\#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `DROP`, `TRUNCATE` y `OPTIMIZE` duplicación de consultas, cuando se ejecuta en `ON CLUSTER` para `ReplicatedMergeTree*` tablas de la familia. [\#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) + +#### Mejora {#improvements-2} + +- Mantener ordinario, `DEFAULT`, `MATERIALIZED` y `ALIAS` columnas en una sola lista (soluciona el problema [\#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [\#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Método de codificación de datos:](https://github.com/ztlpn)) + +### Lanzamiento de ClickHouse 19.4.3.11, 2019-04-02 {#clickhouse-release-19-4-3-11-2019-04-02} + +#### Corrección de errores {#bug-fixes-8} + +- Fix crash en `FULL/RIGHT JOIN` cuando nos unimos en nullable vs no nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar falla de segmentación en `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([propulsor](https://github.com/proller)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-11} + +- Agregue una forma de iniciar una imagen de clickhouse-server desde un usuario personalizado. [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) + +### Lanzamiento de ClickHouse 19.4.2.7, 2019-03-30 {#clickhouse-release-19-4-2-7-2019-03-30} + +#### Corrección de errores {#bug-fixes-9} + +- Lectura fija de `Array(LowCardinality)` columna en caso raro cuando la columna contenía una larga secuencia de matrices vacías. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +### Lanzamiento de ClickHouse 19.4.1.3, 2019-03-19 {#clickhouse-release-19-4-1-3-2019-03-19} + +#### Corrección de errores {#bug-fixes-10} + +- Consultas remotas fijas que contienen ambos `LIMIT BY` y `LIMIT`. Anteriormente, si `LIMIT BY` y `LIMIT` se utilizaron para la consulta remota, `LIMIT` podría pasar antes `LIMIT BY`, cuál llevó al resultado demasiado filtrado. [\#4708](https://github.com/ClickHouse/ClickHouse/pull/4708) ([Constantin S. Pan](https://github.com/kvap)) + +### Lanzamiento de ClickHouse 19.4.0.49, 2019-03-09 {#clickhouse-release-19-4-0-49-2019-03-09} + +#### Novedad {#new-features-5} + +- Se agregó soporte completo para `Protobuf` formato (entrada y salida, estructuras de datos anidadas). [\#4174](https://github.com/ClickHouse/ClickHouse/pull/4174) [\#4493](https://github.com/ClickHouse/ClickHouse/pull/4493) ([Vitaly Baranov](https://github.com/vitlibar)) +- Se agregaron funciones de mapa de bits con mapas de bits rugientes. [\#4207](https://github.com/ClickHouse/ClickHouse/pull/4207) ([Cristina Andrés](https://github.com/andyyzh)) [\#4568](https://github.com/ClickHouse/ClickHouse/pull/4568) ([Vitaly Baranov](https://github.com/vitlibar)) +- Soporte de formato de parquet. [\#4448](https://github.com/ClickHouse/ClickHouse/pull/4448) ([propulsor](https://github.com/proller)) +- Se agregó una distancia de N-gramo para la comparación de cadenas difusas. Es similar a las métricas de q-gram en el lenguaje R. [\#4466](https://github.com/ClickHouse/ClickHouse/pull/4466) ([Más información](https://github.com/danlark1)) +- Combine reglas para el paquete acumulativo de grafito a partir de patrones de agregación y retención dedicados. [\#4426](https://github.com/ClickHouse/ClickHouse/pull/4426) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Añadir `max_execution_speed` y `max_execution_speed_bytes` para limitar el uso de recursos. Añadir `min_execution_speed_bytes` para complementar el `min_execution_speed`. [\#4430](https://github.com/ClickHouse/ClickHouse/pull/4430) ([Invierno Zhang](https://github.com/zhang2014)) +- Función implementada `flatten`. [\#4555](https://github.com/ClickHouse/ClickHouse/pull/4555) [\#4409](https://github.com/ClickHouse/ClickHouse/pull/4409) ([alexey-milovidov](https://github.com/alexey-milovidov), [Kzon](https://github.com/kzon)) +- Funciones añadidas `arrayEnumerateDenseRanked` y `arrayEnumerateUniqRanked` (es como `arrayEnumerateUniq` pero permite ajustar la profundidad de la matriz para mirar dentro de las matrices multidimensionales). [\#4475](https://github.com/ClickHouse/ClickHouse/pull/4475) ([propulsor](https://github.com/proller)) [\#4601](https://github.com/ClickHouse/ClickHouse/pull/4601) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Multiple JOINS with some restrictions: no asterisks, no complex aliases in ON/WHERE/GROUP BY/… [\#4462](https://github.com/ClickHouse/ClickHouse/pull/4462) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Corrección de errores {#bug-fixes-11} + +- Esta versión también contiene todas las correcciones de errores de 19.3 y 19.1. +- Se corrigió un error en los índices de omisión de datos: el orden de los gránulos después de INSERTAR era incorrecto. [\#4407](https://github.com/ClickHouse/ClickHouse/pull/4407) ([Nikita Vasilev](https://github.com/nikvas0)) +- Fijo `set` índice de `Nullable` y `LowCardinality` columna. Antes de eso, `set` índice con `Nullable` o `LowCardinality` columna llevó a error `Data type must be deserialized with multiple streams` mientras se selecciona. [\#4594](https://github.com/ClickHouse/ClickHouse/pull/4594) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Establecer correctamente update\_time en completo `executable` actualización del diccionario. [\#4551](https://github.com/ClickHouse/ClickHouse/pull/4551) ([Tema Novikov](https://github.com/temoon)) +- Arreglar la barra de progreso rota en 19.3. [\#4627](https://github.com/ClickHouse/ClickHouse/pull/4627) ([filimonov](https://github.com/filimonov)) +- Se corrigieron los valores inconsistentes de MemoryTracker cuando se redujo la región de la memoria, en ciertos casos. [\#4619](https://github.com/ClickHouse/ClickHouse/pull/4619) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Comportamiento indefinido fijo en ThreadPool. [\#4612](https://github.com/ClickHouse/ClickHouse/pull/4612) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió un accidente muy raro con el mensaje `mutex lock failed: Invalid argument` eso podría suceder cuando una tabla MergeTree se eliminó simultáneamente con un SELECT. [\#4608](https://github.com/ClickHouse/ClickHouse/pull/4608) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Compatibilidad del controlador ODBC con `LowCardinality` tipo de datos. [\#4381](https://github.com/ClickHouse/ClickHouse/pull/4381) ([propulsor](https://github.com/proller)) +- FreeBSD: Arreglo para `AIOcontextPool: Found io_event with unknown id 0` error. [\#4438](https://github.com/ClickHouse/ClickHouse/pull/4438) ([urgordeadbeef](https://github.com/urgordeadbeef)) +- `system.part_log` se creó independientemente de la configuración. [\#4483](https://github.com/ClickHouse/ClickHouse/pull/4483) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Repara el comportamiento indefinido en `dictIsIn` función para los diccionarios de caché. [\#4515](https://github.com/ClickHouse/ClickHouse/pull/4515) ([alesapin](https://github.com/alesapin)) +- Fixed a deadlock when a SELECT query locks the same table multiple times (e.g. from different threads or when executing multiple subqueries) and there is a concurrent DDL query. [\#4535](https://github.com/ClickHouse/ClickHouse/pull/4535) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Deshabilite compile\_expressions de forma predeterminada hasta que obtengamos el propio `llvm` contrib y puede probarlo con `clang` y `asan`. [\#4579](https://github.com/ClickHouse/ClickHouse/pull/4579) ([alesapin](https://github.com/alesapin)) +- Prevenir `std::terminate` cuando `invalidate_query` para `clickhouse` fuente de diccionario externo ha devuelto un conjunto de resultados incorrecto (vacío o más de una fila o más de una columna). Solucionado el problema cuando `invalidate_query` se realizó cada cinco segundos independientemente de la `lifetime`. [\#4583](https://github.com/ClickHouse/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite el punto muerto cuando `invalidate_query` para un diccionario con `clickhouse` fuente estaba involucrando `system.dictionaries` mesa o `Dictionaries` base de datos (caso raro). [\#4599](https://github.com/ClickHouse/ClickHouse/pull/4599) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Correcciones para CROSS JOIN con WHERE vacío. [\#4598](https://github.com/ClickHouse/ClickHouse/pull/4598) ([Artem Zuikov](https://github.com/4ertus2)) +- Segfault fijo en la función “replicate” cuando se pasa el argumento constante. [\#4603](https://github.com/ClickHouse/ClickHouse/pull/4603) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Repara la función lambda con el optimizador de predicados. [\#4408](https://github.com/ClickHouse/ClickHouse/pull/4408) ([Invierno Zhang](https://github.com/zhang2014)) +- Múltiples JOINs múltiples correcciones. [\#4595](https://github.com/ClickHouse/ClickHouse/pull/4595) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejora {#improvements-3} + +- Soporte alias en la sección JOIN ON para columnas de la tabla derecha. [\#4412](https://github.com/ClickHouse/ClickHouse/pull/4412) ([Artem Zuikov](https://github.com/4ertus2)) +- El resultado de múltiples JOIN necesita nombres de resultados correctos para ser utilizados en subselecciones. Reemplace los alias planos con nombres de origen en el resultado. [\#4474](https://github.com/ClickHouse/ClickHouse/pull/4474) ([Artem Zuikov](https://github.com/4ertus2)) +- Mejorar la lógica push-down para sentencias unidas. [\#4387](https://github.com/ClickHouse/ClickHouse/pull/4387) ([Ivan](https://github.com/abyss7)) + +#### Mejoras de rendimiento {#performance-improvements-3} + +- Heurística mejorada de “move to PREWHERE” optimización. [\#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Use tablas de búsqueda adecuadas que usen la API de HashTable para claves de 8 bits y 16 bits. [\#4536](https://github.com/ClickHouse/ClickHouse/pull/4536) ([Amos pájaro](https://github.com/amosbird)) +- Mejora del rendimiento de la comparación de cadenas. [\#4564](https://github.com/ClickHouse/ClickHouse/pull/4564) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Limpie la cola DDL distribuida en un subproceso separado para que no ralentice el bucle principal que procesa las tareas DDL distribuidas. [\#4502](https://github.com/ClickHouse/ClickHouse/pull/4502) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Cuando `min_bytes_to_use_direct_io` se establece en 1, no todos los archivos se abrieron con el modo O\_DIRECT porque el tamaño de los datos a leer a veces se subestimó por el tamaño de un bloque comprimido. [\#4526](https://github.com/ClickHouse/ClickHouse/pull/4526) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-12} + +- Se agregó soporte para clang-9 [\#4604](https://github.com/ClickHouse/ClickHouse/pull/4604) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Arreglar mal `__asm__` instrucciones (de nuevo) [\#4621](https://github.com/ClickHouse/ClickHouse/pull/4621) ([Konstantin Podshumok](https://github.com/podshumok)) +- Añadir capacidad para especificar la configuración para `clickhouse-performance-test` desde la línea de comandos. [\#4437](https://github.com/ClickHouse/ClickHouse/pull/4437) ([alesapin](https://github.com/alesapin)) +- Agregue pruebas de diccionarios a las pruebas de integración. [\#4477](https://github.com/ClickHouse/ClickHouse/pull/4477) ([alesapin](https://github.com/alesapin)) +- Se agregaron consultas desde el punto de referencia en el sitio web a pruebas de rendimiento automatizadas. [\#4496](https://github.com/ClickHouse/ClickHouse/pull/4496) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `xxhash.h` no existe en lz4 externo porque es un detalle de implementación y sus símbolos tienen un espacio de nombres con `XXH_NAMESPACE` macro. Cuando lz4 es externo, xxHash también tiene que ser externo, y los dependientes tienen que vincularlo. [\#4495](https://github.com/ClickHouse/ClickHouse/pull/4495) ([Películas De Sexo](https://github.com/orivej)) +- Se corrigió un caso cuando `quantileTiming` se puede llamar a la función de agregado con argumento de punto negativo o flotante (esto corrige la prueba de fuzz con un desinfectante de comportamiento indefinido). [\#4506](https://github.com/ClickHouse/ClickHouse/pull/4506) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corrección de errores de ortografía. [\#4531](https://github.com/ClickHouse/ClickHouse/pull/4531) ([sdk2](https://github.com/sdk2)) +- Corregir la compilación en Mac. [\#4371](https://github.com/ClickHouse/ClickHouse/pull/4371) ([Vitaly Baranov](https://github.com/vitlibar)) +- Construir correcciones para FreeBSD y varias configuraciones de compilación inusuales. [\#4444](https://github.com/ClickHouse/ClickHouse/pull/4444) ([propulsor](https://github.com/proller)) + +## Lanzamiento de ClickHouse 19.3 {#clickhouse-release-19-3} + +### Lanzamiento de ClickHouse 19.3.9.1, 2019-04-02 {#clickhouse-release-19-3-9-1-2019-04-02} + +#### Corrección de errores {#bug-fixes-12} + +- Fix crash en `FULL/RIGHT JOIN` cuando nos unimos en nullable vs no nullable. [\#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) +- Fijar falla de segmentación en `clickhouse-copier`. [\#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([propulsor](https://github.com/proller)) +- Lectura fija de `Array(LowCardinality)` columna en caso raro cuando la columna contenía una larga secuencia de matrices vacías. [\#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-13} + +- Agregue una forma de iniciar una imagen de clickhouse-server desde un usuario personalizado [\#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) + +### Lanzamiento de ClickHouse 19.3.7, 2019-03-12 {#clickhouse-release-19-3-7-2019-03-12} + +#### Corrección de errores {#bug-fixes-13} + +- Corregido el error en \# 3920. Este error se manifiesta como corrupción de caché aleatoria (mensajes `Unknown codec family code`, `Cannot seek through file`) y segfaults. Este error apareció por primera vez en la versión 19.1 y está presente en las versiones hasta 19.1.10 y 19.3.6. [\#4623](https://github.com/ClickHouse/ClickHouse/pull/4623) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.3.6, 2019-03-02 {#clickhouse-release-19-3-6-2019-03-02} + +#### Corrección de errores {#bug-fixes-14} + +- Cuando hay más de 1000 subprocesos en un grupo de subprocesos, `std::terminate` puede suceder en la salida del hilo. [Azat Khuzhin](https://github.com/azat) [\#4485](https://github.com/ClickHouse/ClickHouse/pull/4485) [\#4505](https://github.com/ClickHouse/ClickHouse/pull/4505) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora es posible crear `ReplicatedMergeTree*` tablas con comentarios sobre columnas sin valores predeterminados y tablas con códecs de columnas sin comentarios y valores predeterminados. También corrige la comparación de códecs. [\#4523](https://github.com/ClickHouse/ClickHouse/pull/4523) ([alesapin](https://github.com/alesapin)) +- Se corrigió el fallo en JOIN con matriz o tupla. [\#4552](https://github.com/ClickHouse/ClickHouse/pull/4552) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el fallo en el clickhouse-copiadora con el mensaje `ThreadStatus not created`. [\#4540](https://github.com/ClickHouse/ClickHouse/pull/4540) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el bloqueo en el cierre del servidor si se usaban DDL distribuidos. [\#4472](https://github.com/ClickHouse/ClickHouse/pull/4472) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Se imprimieron números de columna incorrectos en un mensaje de error sobre el análisis de formato de texto para columnas con un número mayor que 10. [\#4484](https://github.com/ClickHouse/ClickHouse/pull/4484) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-3} + +- Compilación fija con AVX habilitado. [\#4527](https://github.com/ClickHouse/ClickHouse/pull/4527) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Habilite la contabilidad extendida y la contabilidad IO basada en una versión buena conocida en lugar del kernel bajo el cual se compila. [\#4541](https://github.com/ClickHouse/ClickHouse/pull/4541) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) +- Permitir omitir la configuración de core\_dump.size\_limit, advertencia en lugar de lanzar si el conjunto de límites falla. [\#4473](https://github.com/ClickHouse/ClickHouse/pull/4473) ([propulsor](https://github.com/proller)) +- Eliminado el `inline` etiquetas de `void readBinary(...)` en `Field.cpp`. También se fusionó redundante `namespace DB` bloque. [\#4530](https://github.com/ClickHouse/ClickHouse/pull/4530) ([Hcz](https://github.com/hczhcz)) + +### Lanzamiento de ClickHouse 19.3.5, 2019-02-21 {#clickhouse-release-19-3-5-2019-02-21} + +#### Corrección de errores {#bug-fixes-15} + +- Se corrigió un error con el procesamiento de grandes consultas de inserción http. [\#4454](https://github.com/ClickHouse/ClickHouse/pull/4454) ([alesapin](https://github.com/alesapin)) +- Se corrigió la incompatibilidad hacia atrás con versiones antiguas debido a una implementación incorrecta de `send_logs_level` configuración. [\#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Incompatibilidad hacia atrás fija de la función de la tabla `remote` introducido con comentarios de columna. [\#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.3.4, 2019-02-16 {#clickhouse-release-19-3-4-2019-02-16} + +#### Mejora {#improvements-4} + +- El tamaño del índice de tabla no se tiene en cuenta los límites de memoria al hacer `ATTACH TABLE` consulta. Evitó la posibilidad de que una tabla no se pueda adjuntar después de haber sido separada. [\#4396](https://github.com/ClickHouse/ClickHouse/pull/4396) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Levantó ligeramente el límite en el tamaño máximo de cadena y matriz recibido de ZooKeeper. Permite continuar trabajando con un mayor tamaño de `CLIENT_JVMFLAGS=-Djute.maxbuffer=...` en ZooKeeper. [\#4398](https://github.com/ClickHouse/ClickHouse/pull/4398) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir reparar réplicas abandonadas incluso si ya tiene una gran cantidad de nodos en su cola. [\#4399](https://github.com/ClickHouse/ClickHouse/pull/4399) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue un argumento requerido a `SET` índice (número máximo de filas almacenadas). [\#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) + +#### Corrección de errores {#bug-fixes-16} + +- Fijo `WITH ROLLUP` resultado para grupo por solo `LowCardinality` clave. [\#4384](https://github.com/ClickHouse/ClickHouse/pull/4384) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se corrigió un error en el índice establecido (dejando caer un gránulo si contiene más de `max_rows` filas). [\#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) +- Muchas correcciones de compilación de FreeBSD. [\#4397](https://github.com/ClickHouse/ClickHouse/pull/4397) ([propulsor](https://github.com/proller)) +- Se corrigió la sustitución de alias en consultas con subconsulta que contenía el mismo alias (problema [\#4110](https://github.com/ClickHouse/ClickHouse/issues/4110)). [\#4351](https://github.com/ClickHouse/ClickHouse/pull/4351) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-4} + +- Añadir capacidad de ejecutar `clickhouse-server` para pruebas sin estado en la imagen de la ventana acoplable. [\#4347](https://github.com/ClickHouse/ClickHouse/pull/4347) ([Vasily Nemkov](https://github.com/Enmk)) + +### Lanzamiento de ClickHouse 19.3.3, 2019-02-13 {#clickhouse-release-19-3-3-2019-02-13} + +#### Novedad {#new-features-6} + +- Se agregó el `KILL MUTATION` declaración que permite eliminar mutaciones que por alguna razón están atascadas. Añadir `latest_failed_part`, `latest_fail_time`, `latest_fail_reason` campos a la `system.mutations` mesa para una solución de problemas más fácil. [\#4287](https://github.com/ClickHouse/ClickHouse/pull/4287) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Función agregada agregada `entropy` que calcula la entropía de Shannon. [\#4238](https://github.com/ClickHouse/ClickHouse/pull/4238) ([Quid37](https://github.com/Quid37)) +- Añadida la capacidad de enviar consultas `INSERT INTO tbl VALUES (....` al servidor sin dividir en `query` y `data` parte. [\#4301](https://github.com/ClickHouse/ClickHouse/pull/4301) ([alesapin](https://github.com/alesapin)) +- Implementación genérica de `arrayWithConstant` se añadió la función. [\#4322](https://github.com/ClickHouse/ClickHouse/pull/4322) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Aplicado `NOT BETWEEN` operador de comparación. [\#4228](https://github.com/ClickHouse/ClickHouse/pull/4228) ([Dmitry Naumov](https://github.com/nezed)) +- Implementar `sumMapFiltered` para poder limitar el número de claves para las cuales los valores se sumarán por `sumMap`. [\#4129](https://github.com/ClickHouse/ClickHouse/pull/4129) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Se agregó soporte de `Nullable` tipos en `mysql` función de la tabla. [\#4198](https://github.com/ClickHouse/ClickHouse/pull/4198) ([Emmanuel Donin de Rosière](https://github.com/edonin)) +- Soporte para expresiones constantes arbitrarias en `LIMIT` clausula. [\#4246](https://github.com/ClickHouse/ClickHouse/pull/4246) ([K3box](https://github.com/k3box)) +- Añadir `topKWeighted` función de agregado que toma argumentos adicionales con peso (entero sin signo). [\#4245](https://github.com/ClickHouse/ClickHouse/pull/4245) ([Córdoba](https://github.com/andrewgolman)) +- `StorageJoin` ahora soporta `join_any_take_last_row` configuración que permite sobrescribir los valores existentes de la misma clave. [\#3973](https://github.com/ClickHouse/ClickHouse/pull/3973) ([Amos pájaro](https://github.com/amosbird) +- Función añadida `toStartOfInterval`. [\#4304](https://github.com/ClickHouse/ClickHouse/pull/4304) ([Vitaly Baranov](https://github.com/vitlibar)) +- Añadir `RowBinaryWithNamesAndTypes` formato. [\#4200](https://github.com/ClickHouse/ClickHouse/pull/4200) ([Oleg V. Kozlyuk](https://github.com/DarkWanderer)) +- Añadir `IPv4` y `IPv6` tipos de datos. Implementaciones más efectivas de `IPv*` función. [\#3669](https://github.com/ClickHouse/ClickHouse/pull/3669) ([Vasily Nemkov](https://github.com/Enmk)) +- Función añadida `toStartOfTenMinutes()`. [\#4298](https://github.com/ClickHouse/ClickHouse/pull/4298) ([Vitaly Baranov](https://github.com/vitlibar)) +- Añadir `Protobuf` formato de salida. [\#4005](https://github.com/ClickHouse/ClickHouse/pull/4005) [\#4158](https://github.com/ClickHouse/ClickHouse/pull/4158) ([Vitaly Baranov](https://github.com/vitlibar)) +- Añadido soporte brotli para la interfaz HTTP para la importación de datos (INSERTs). [\#4235](https://github.com/ClickHouse/ClickHouse/pull/4235) ([Mijaíl](https://github.com/fandyushin)) +- Se agregaron consejos mientras el usuario hace un error tipográfico en el nombre de la función o escribe el cliente de línea de comandos. [\#4239](https://github.com/ClickHouse/ClickHouse/pull/4239) ([Más información](https://github.com/danlark1)) +- Añadir `Query-Id` al encabezado de respuesta HTTP del servidor. [\#4231](https://github.com/ClickHouse/ClickHouse/pull/4231) ([Mijaíl](https://github.com/fandyushin)) + +#### Experimental características {#experimental-features-2} + +- Añadir `minmax` y `set` Índices de saltos de datos para la familia de motores de tablas MergeTree. [\#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) +- Añadido conversión de `CROSS JOIN` a `INNER JOIN` si es posible. [\#4221](https://github.com/ClickHouse/ClickHouse/pull/4221) [\#4266](https://github.com/ClickHouse/ClickHouse/pull/4266) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Corrección de errores {#bug-fixes-17} + +- Fijo `Not found column` para columnas duplicadas en `JOIN ON` apartado. [\#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) +- Hacer `START REPLICATED SENDS` comando iniciar envíos replicados. [\#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) +- Ejecución de funciones agregadas fijas con `Array(LowCardinality)` argumento. [\#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Corregido el comportamiento incorrecto al hacer `INSERT ... SELECT ... FROM file(...)` consulta y archivo `CSVWithNames` o `TSVWIthNames` formato y falta la primera fila de datos. [\#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el fallo en la recarga del diccionario si el diccionario no está disponible. Este error apareció en 19.1.6. [\#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([propulsor](https://github.com/proller)) +- Fijo `ALL JOIN` con duplicados en la tabla derecha. [\#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) +- Fallo de segmentación fijo con `use_uncompressed_cache=1` y excepción con un tamaño incorrecto sin comprimir. Este error apareció en 19.1.6. [\#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) +- Fijo `compile_expressions` error con la comparación de fechas grandes (más que int16). [\#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) +- Bucle infinito fijo al seleccionar de la función de la tabla `numbers(0)`. [\#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deshabilitar temporalmente la optimización de predicados para `ORDER BY`. [\#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Invierno Zhang](https://github.com/zhang2014)) +- Fijo `Illegal instruction` error al usar funciones base64 en CPU antiguas. Este error se ha reproducido solo cuando ClickHouse se compiló con gcc-8. [\#4275](https://github.com/ClickHouse/ClickHouse/pull/4275) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijo `No message received` error al interactuar con PostgreSQL ODBC Driver a través de la conexión TLS. También corrige segfault cuando se utiliza MySQL ODBC Driver. [\#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el resultado incorrecto cuando `Date` y `DateTime` los argumentos se usan en ramas del operador condicional (función `if`). Añadido caso genérico para la función `if`. [\#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Los diccionarios de ClickHouse ahora se cargan dentro `clickhouse` proceso. [\#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el punto muerto cuando `SELECT` de una mesa con `File` el motor fue reintentado después `No such file or directory` error. [\#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Condición de carrera fija al seleccionar entre `system.tables` puede dar `table doesn't exist` error. [\#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `clickhouse-client` puede segfault al salir mientras carga datos para sugerencias de línea de comandos si se ejecutó en modo interactivo. [\#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha corregido un error cuando la ejecución de mutaciones que contienen `IN` Los operadores estaban produciendo resultados incorrectos. [\#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Error corregido: si hay una base de datos con `Dictionary` motor, todos los diccionarios obligados a cargar en el inicio del servidor, y si hay un diccionario con fuente ClickHouse de localhost, el diccionario no se puede cargar. [\#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el error cuando los registros del sistema se intentaban crear de nuevo al apagar el servidor. [\#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Devuelva correctamente el tipo correcto y maneje adecuadamente las cerraduras en `joinGet` función. [\#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos pájaro](https://github.com/amosbird)) +- Añadir `sumMapWithOverflow` función. [\#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Segfault fijo con `allow_experimental_multiple_joins_emulation`. [52de2c](https://github.com/ClickHouse/ClickHouse/commit/52de2cd927f7b5257dd67e175f0a5560a48840d0) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregido error con incorrecto `Date` y `DateTime` comparación. [\#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([Valexey](https://github.com/valexey)) +- Prueba de fuzz fija bajo desinfectante de comportamiento indefinido: verificación de tipo de parámetro agregada para `quantile*Weighted` familia de funciones. [\#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la condición de carrera rara al eliminar las piezas de datos antiguas que pueden fallar con `File not found` error. [\#4378](https://github.com/ClickHouse/ClickHouse/pull/4378) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Arregle el paquete de instalación con /etc/clickhouse-server/config que falta.XML. [\#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([propulsor](https://github.com/proller)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-5} + +- Paquete de Debian: correcto /etc/clickhouse-server/preprocessed link según config. [\#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([propulsor](https://github.com/proller)) +- Varias correcciones de compilación para FreeBSD. [\#4225](https://github.com/ClickHouse/ClickHouse/pull/4225) ([propulsor](https://github.com/proller)) +- Se agregó la capacidad de crear, rellenar y soltar tablas en perftest. [\#4220](https://github.com/ClickHouse/ClickHouse/pull/4220) ([alesapin](https://github.com/alesapin)) +- Se ha añadido un script para comprobar si hay duplicados incluye. [\#4326](https://github.com/ClickHouse/ClickHouse/pull/4326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó la capacidad de ejecutar consultas por índice en la prueba de rendimiento. [\#4264](https://github.com/ClickHouse/ClickHouse/pull/4264) ([alesapin](https://github.com/alesapin)) +- Se sugiere instalar un paquete con símbolos de depuración. [\#4274](https://github.com/ClickHouse/ClickHouse/pull/4274) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Refactorización de pruebas de rendimiento. Mejor registro y manejo de señales. [\#4171](https://github.com/ClickHouse/ClickHouse/pull/4171) ([alesapin](https://github.com/alesapin)) +- Se agregaron documentos a Yandex anónimo.Conjuntos de datos Metrika. [\#4164](https://github.com/ClickHouse/ClickHouse/pull/4164) ([alesapin](https://github.com/alesapin)) +- Аdded tool for converting an old month-partitioned part to the custom-partitioned format. [\#4195](https://github.com/ClickHouse/ClickHouse/pull/4195) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Se agregaron documentos sobre dos conjuntos de datos en s3. [\#4144](https://github.com/ClickHouse/ClickHouse/pull/4144) ([alesapin](https://github.com/alesapin)) +- Se agregó un script que crea un registro de cambios a partir de la descripción de las solicitudes de extracción. [\#4169](https://github.com/ClickHouse/ClickHouse/pull/4169) [\#4173](https://github.com/ClickHouse/ClickHouse/pull/4173) ([KochetovNicolai](https://github.com/KochetovNicolai)) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Se agregó un módulo de marionetas para Clickhouse. [\#4182](https://github.com/ClickHouse/ClickHouse/pull/4182) ([Maxim Fedotov](https://github.com/MaxFedotov)) +- Se agregaron documentos para un grupo de funciones no documentadas. [\#4168](https://github.com/ClickHouse/ClickHouse/pull/4168) ([Invierno Zhang](https://github.com/zhang2014)) +- ARM correcciones de construcción. [\#4210](https://github.com/ClickHouse/ClickHouse/pull/4210)[\#4306](https://github.com/ClickHouse/ClickHouse/pull/4306) [\#4291](https://github.com/ClickHouse/ClickHouse/pull/4291) ([propulsor](https://github.com/proller)) ([propulsor](https://github.com/proller)) +- Las pruebas de diccionario ahora pueden ejecutarse desde `ctest`. [\#4189](https://github.com/ClickHouse/ClickHouse/pull/4189) ([propulsor](https://github.com/proller)) +- Ahora `/etc/ssl` se utiliza como directorio predeterminado con certificados SSL. [\#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó la comprobación de las instrucciones SSE y AVX al inicio. [\#4234](https://github.com/ClickHouse/ClickHouse/pull/4234) ([Igr](https://github.com/igron99)) +- La secuencia de comandos Init esperará el servidor hasta el inicio. [\#4281](https://github.com/ClickHouse/ClickHouse/pull/4281) ([propulsor](https://github.com/proller)) + +#### Cambios incompatibles hacia atrás {#backward-incompatible-changes-1} + +- Quitar `allow_experimental_low_cardinality_type` configuración. `LowCardinality` los tipos de datos están listos para la producción. [\#4323](https://github.com/ClickHouse/ClickHouse/pull/4323) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reduzca el tamaño de la caché de marca y el tamaño de la caché sin comprimir según la cantidad de memoria disponible. [\#4240](https://github.com/ClickHouse/ClickHouse/pull/4240) ([Lopatin Konstantin](https://github.com/k-lopatin) +- Palabra clave añadida `INDEX` en `CREATE TABLE` consulta. Una columna con nombre `index` debe citarse con retrocesos o comillas dobles: `` `index` ``. [\#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) +- `sumMap` ahora promueve el tipo de resultado en lugar de desbordamiento. Antiguo `sumMap` se puede obtener mediante el uso de `sumMapWithOverflow` función. [\#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) + +#### Mejoras de rendimiento {#performance-improvements-4} + +- `std::sort` sustituido por `pdqsort` para consultas sin `LIMIT`. [\#4236](https://github.com/ClickHouse/ClickHouse/pull/4236) ([Evgenii Pravda](https://github.com/kvinty)) +- Ahora el servidor reutiliza subprocesos del grupo de subprocesos global. Esto afecta el rendimiento en algunos casos de esquina. [\#4150](https://github.com/ClickHouse/ClickHouse/pull/4150) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora {#improvements-5} + +- Implementado soporte AIO para FreeBSD. [\#4305](https://github.com/ClickHouse/ClickHouse/pull/4305) ([urgordeadbeef](https://github.com/urgordeadbeef)) +- `SELECT * FROM a JOIN b USING a, b` ahora volver `a` y `b` solo de la tabla de la izquierda. [\#4141](https://github.com/ClickHouse/ClickHouse/pull/4141) ([Artem Zuikov](https://github.com/4ertus2)) +- Permitir `-C` opción del cliente para trabajar como `-c` opcion. [\#4232](https://github.com/ClickHouse/ClickHouse/pull/4232) ([syominsergey](https://github.com/syominsergey)) +- Ahora opción `--password` utilizado sin valor requiere contraseña de stdin. [\#4230](https://github.com/ClickHouse/ClickHouse/pull/4230) ([BSD\_Conqueror](https://github.com/bsd-conqueror)) +- Se agregó resaltado de metacaracteres no escapados en literales de cadena que contienen `LIKE` expresiones o regex. [\#4327](https://github.com/ClickHouse/ClickHouse/pull/4327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó la cancelación de las consultas de solo lectura HTTP si el socket del cliente desaparece. [\#4213](https://github.com/ClickHouse/ClickHouse/pull/4213) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) +- Ahora los informes del servidor progresan para mantener vivas las conexiones del cliente. [\#4215](https://github.com/ClickHouse/ClickHouse/pull/4215) ([Ivan](https://github.com/abyss7)) +- Mensaje ligeramente mejor con motivo para OPTIMIZE consulta con `optimize_throw_if_noop` configuración activada. [\#4294](https://github.com/ClickHouse/ClickHouse/pull/4294) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó soporte de `--version` opción para el servidor de clickhouse. [\#4251](https://github.com/ClickHouse/ClickHouse/pull/4251) ([Lopatin Konstantin](https://github.com/k-lopatin)) +- Añadir `--help/-h` opción para `clickhouse-server`. [\#4233](https://github.com/ClickHouse/ClickHouse/pull/4233) ([Yuriy Baranov](https://github.com/yurriy)) +- Se agregó soporte para subconsultas escalares con el resultado del estado de la función agregada. [\#4348](https://github.com/ClickHouse/ClickHouse/pull/4348) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Mejora del tiempo de apagado del servidor y ALTERa el tiempo de espera. [\#4372](https://github.com/ClickHouse/ClickHouse/pull/4372) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha añadido información sobre la configuración de la línea de replicado\_can\_become\_leader en el sistema.réplicas y agregue registro si la réplica no intenta convertirse en líder. [\#4379](https://github.com/ClickHouse/ClickHouse/pull/4379) ([Método de codificación de datos:](https://github.com/ztlpn)) + +## Lanzamiento de ClickHouse 19.1 {#clickhouse-release-19-1} + +### Lanzamiento de ClickHouse 19.1.14, 2019-03-14 {#clickhouse-release-19-1-14-2019-03-14} + +- Error fijo `Column ... queried more than once` que puede suceder si la configuración `asterisk_left_columns_only` se establece en 1 en caso de usar `GLOBAL JOIN` con `SELECT *` (caso raro). El problema no existe en 19.3 y posteriores. [6bac7d8d](https://github.com/ClickHouse/ClickHouse/pull/4692/commits/6bac7d8d11a9b0d6de0b32b53c47eb2f6f8e7062) ([Artem Zuikov](https://github.com/4ertus2)) + +### Lanzamiento de ClickHouse 19.1.13, 2019-03-12 {#clickhouse-release-19-1-13-2019-03-12} + +Esta versión contiene exactamente el mismo conjunto de parches que 19.3.7. + +### Lanzamiento de ClickHouse 19.1.10, 2019-03-03 {#clickhouse-release-19-1-10-2019-03-03} + +Esta versión contiene exactamente el mismo conjunto de parches que 19.3.6. + +## Lanzamiento de ClickHouse 19.1 {#clickhouse-release-19-1-1} + +### Lanzamiento de ClickHouse 19.1.9, 2019-02-21 {#clickhouse-release-19-1-9-2019-02-21} + +#### Corrección de errores {#bug-fixes-18} + +- Se corrigió la incompatibilidad hacia atrás con versiones antiguas debido a una implementación incorrecta de `send_logs_level` configuración. [\#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Incompatibilidad hacia atrás fija de la función de la tabla `remote` introducido con comentarios de columna. [\#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.1.8, 2019-02-16 {#clickhouse-release-19-1-8-2019-02-16} + +#### Corrección de errores {#bug-fixes-19} + +- Arregle el paquete de instalación con /etc/clickhouse-server/config que falta.XML. [\#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([propulsor](https://github.com/proller)) + +## Lanzamiento de ClickHouse 19.1 {#clickhouse-release-19-1-2} + +### Lanzamiento de ClickHouse 19.1.7, 2019-02-15 {#clickhouse-release-19-1-7-2019-02-15} + +#### Corrección de errores {#bug-fixes-20} + +- Devuelva correctamente el tipo correcto y maneje adecuadamente las cerraduras en `joinGet` función. [\#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos pájaro](https://github.com/amosbird)) +- Se corrigió el error cuando los registros del sistema se intentaban crear de nuevo al apagar el servidor. [\#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Error corregido: si hay una base de datos con `Dictionary` motor, todos los diccionarios obligados a cargar en el inicio del servidor, y si hay un diccionario con fuente ClickHouse de localhost, el diccionario no se puede cargar. [\#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha corregido un error cuando la ejecución de mutaciones que contienen `IN` Los operadores estaban produciendo resultados incorrectos. [\#4099](https://github.com/ClickHouse/ClickHouse/pull/4099) ([Método de codificación de datos:](https://github.com/ztlpn)) +- `clickhouse-client` puede segfault al salir mientras carga datos para sugerencias de línea de comandos si se ejecutó en modo interactivo. [\#4317](https://github.com/ClickHouse/ClickHouse/pull/4317) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Condición de carrera fija al seleccionar entre `system.tables` puede dar `table doesn't exist` error. [\#4313](https://github.com/ClickHouse/ClickHouse/pull/4313) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el punto muerto cuando `SELECT` de una mesa con `File` el motor fue reintentado después `No such file or directory` error. [\#4161](https://github.com/ClickHouse/ClickHouse/pull/4161) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha solucionado un problema: los diccionarios locales de ClickHouse se cargan a través de TCP, pero deberían cargarse dentro del proceso. [\#4166](https://github.com/ClickHouse/ClickHouse/pull/4166) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijo `No message received` error al interactuar con PostgreSQL ODBC Driver a través de la conexión TLS. También corrige segfault cuando se utiliza MySQL ODBC Driver. [\#4170](https://github.com/ClickHouse/ClickHouse/pull/4170) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Deshabilitar temporalmente la optimización de predicados para `ORDER BY`. [\#3890](https://github.com/ClickHouse/ClickHouse/pull/3890) ([Invierno Zhang](https://github.com/zhang2014)) +- Bucle infinito fijo al seleccionar de la función de la tabla `numbers(0)`. [\#4280](https://github.com/ClickHouse/ClickHouse/pull/4280) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijo `compile_expressions` error con la comparación de fechas grandes (más que int16). [\#4341](https://github.com/ClickHouse/ClickHouse/pull/4341) ([alesapin](https://github.com/alesapin)) +- Fallo de segmentación fijo con `uncompressed_cache=1` y excepción con un tamaño incorrecto sin comprimir. [\#4186](https://github.com/ClickHouse/ClickHouse/pull/4186) ([alesapin](https://github.com/alesapin)) +- Fijo `ALL JOIN` con duplicados en la tabla derecha. [\#4184](https://github.com/ClickHouse/ClickHouse/pull/4184) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregido el comportamiento incorrecto al hacer `INSERT ... SELECT ... FROM file(...)` consulta y archivo `CSVWithNames` o `TSVWIthNames` formato y falta la primera fila de datos. [\#4297](https://github.com/ClickHouse/ClickHouse/pull/4297) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ejecución de funciones agregadas fijas con `Array(LowCardinality)` argumento. [\#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Paquete de Debian: correcto /etc/clickhouse-server/preprocessed link según config. [\#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([propulsor](https://github.com/proller)) +- Prueba de fuzz fija bajo desinfectante de comportamiento indefinido: verificación de tipo de parámetro agregada para `quantile*Weighted` familia de funciones. [\#4145](https://github.com/ClickHouse/ClickHouse/pull/4145) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Hacer `START REPLICATED SENDS` comando iniciar envíos replicados. [\#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([Nombre de la red inalámbrica (SSID):](https://github.com/nvartolomei)) +- Fijo `Not found column` para columnas duplicadas en la sección JOIN ON. [\#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) +- Ahora `/etc/ssl` se utiliza como directorio predeterminado con certificados SSL. [\#4167](https://github.com/ClickHouse/ClickHouse/pull/4167) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el fallo en la recarga del diccionario si el diccionario no está disponible. [\#4188](https://github.com/ClickHouse/ClickHouse/pull/4188) ([propulsor](https://github.com/proller)) +- Corregido error con incorrecto `Date` y `DateTime` comparación. [\#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([Valexey](https://github.com/valexey)) +- Se corrigió el resultado incorrecto cuando `Date` y `DateTime` los argumentos se usan en ramas del operador condicional (función `if`). Añadido caso genérico para la función `if`. [\#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Lanzamiento de ClickHouse 19.1.6, 2019-01-24 {#clickhouse-release-19-1-6-2019-01-24} + +#### Novedad {#new-features-7} + +- Códecs de compresión personalizados por columna para tablas. [\#3899](https://github.com/ClickHouse/ClickHouse/pull/3899) [\#4111](https://github.com/ClickHouse/ClickHouse/pull/4111) ([alesapin](https://github.com/alesapin), [Invierno Zhang](https://github.com/zhang2014), [Anatoly](https://github.com/Sindbag)) +- Añadido codec de compresión `Delta`. [\#4052](https://github.com/ClickHouse/ClickHouse/pull/4052) ([alesapin](https://github.com/alesapin)) +- Permitir a `ALTER` códecs de compresión. [\#4054](https://github.com/ClickHouse/ClickHouse/pull/4054) ([alesapin](https://github.com/alesapin)) +- Funciones añadidas `left`, `right`, `trim`, `ltrim`, `rtrim`, `timestampadd`, `timestampsub` para la compatibilidad estándar SQL. [\#3826](https://github.com/ClickHouse/ClickHouse/pull/3826) ([Ivan Blinkov](https://github.com/blinkov)) +- Soporte para escribir en `HDFS` mesas y `hdfs` función de la tabla. [\#4084](https://github.com/ClickHouse/ClickHouse/pull/4084) ([alesapin](https://github.com/alesapin)) +- Funciones añadidas para buscar múltiples cadenas constantes de gran pajar: `multiPosition`, `multiSearch` ,`firstMatch` también con `-UTF8`, `-CaseInsensitive`, y `-CaseInsensitiveUTF8` variante. [\#4053](https://github.com/ClickHouse/ClickHouse/pull/4053) ([Más información](https://github.com/danlark1)) +- Poda de fragmentos no utilizados si `SELECT` filtros de consulta por clave sharding (configuración `optimize_skip_unused_shards`). [\#3851](https://github.com/ClickHouse/ClickHouse/pull/3851) ([Gleb Kanterov](https://github.com/kanterov), [Ivan](https://github.com/abyss7)) +- Permitir `Kafka` para ignorar cierta cantidad de errores de análisis por bloque. [\#4094](https://github.com/ClickHouse/ClickHouse/pull/4094) ([Ivan](https://github.com/abyss7)) +- Añadido soporte para `CatBoost` evaluación de modelos multiclase. Función `modelEvaluate` devuelve tupla con predicciones sin procesar por clase para modelos multiclase. `libcatboostmodel.so` debe ser construido con [\#607](https://github.com/catboost/catboost/pull/607). [\#3959](https://github.com/ClickHouse/ClickHouse/pull/3959) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Funciones añadidas `filesystemAvailable`, `filesystemFree`, `filesystemCapacity`. [\#4097](https://github.com/ClickHouse/ClickHouse/pull/4097) ([Boris Granveaud](https://github.com/bgranvea)) +- Funciones hash añadidas `xxHash64` y `xxHash32`. [\#3905](https://github.com/ClickHouse/ClickHouse/pull/3905) ([filimonov](https://github.com/filimonov)) +- Añadir `gccMurmurHash` función de hash (hash de Murmur con sabor a GCC) que usa la misma semilla de hash que [Gcc](https://github.com/gcc-mirror/gcc/blob/41d6b10e96a1de98e90a7c0378437c3255814b16/libstdc%2B%2B-v3/include/bits/functional_hash.h#L191) [\#4000](https://github.com/ClickHouse/ClickHouse/pull/4000) ([Sundyli](https://github.com/sundy-li)) +- Funciones hash añadidas `javaHash`, `hiveHash`. [\#3811](https://github.com/ClickHouse/ClickHouse/pull/3811) ([shangshujie365](https://github.com/shangshujie365)) +- Función de tabla añadida `remoteSecure`. Funciona como `remote`, pero usa una conexión segura. [\#4088](https://github.com/ClickHouse/ClickHouse/pull/4088) ([propulsor](https://github.com/proller)) + +#### Experimental características {#experimental-features-3} + +- Se agregaron múltiples emulaciones JOINs (`allow_experimental_multiple_joins_emulation` configuración). [\#3946](https://github.com/ClickHouse/ClickHouse/pull/3946) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Corrección de errores {#bug-fixes-21} + +- Hacer `compiled_expression_cache_size` ajuste limitado por defecto para reducir el consumo de memoria. [\#4041](https://github.com/ClickHouse/ClickHouse/pull/4041) ([alesapin](https://github.com/alesapin)) +- Se corrigió un error que provocaba bloqueos en los subprocesos que realizaban ALTERs de tablas replicadas y en el subproceso que actualizaba la configuración de ZooKeeper. [\#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [\#3891](https://github.com/ClickHouse/ClickHouse/issues/3891) [\#3934](https://github.com/ClickHouse/ClickHouse/pull/3934) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Se corrigió una condición de carrera al ejecutar una tarea ALTER distribuida. La condición de carrera provocó que más de una réplica intentara ejecutar la tarea y todas las réplicas, excepto una que fallara con un error de ZooKeeper. [\#3904](https://github.com/ClickHouse/ClickHouse/pull/3904) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Corregir un error cuando `from_zk` Los elementos de configuración no se actualizaron después de que se agotó el tiempo de espera de una solicitud a ZooKeeper. [\#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [\#3947](https://github.com/ClickHouse/ClickHouse/pull/3947) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Corregir un error con el prefijo incorrecto para las máscaras de subred IPv4. [\#3945](https://github.com/ClickHouse/ClickHouse/pull/3945) ([alesapin](https://github.com/alesapin)) +- Se corrigió el fallo (`std::terminate`) en casos excepcionales cuando no se puede crear un nuevo subproceso debido a recursos agotados. [\#3956](https://github.com/ClickHouse/ClickHouse/pull/3956) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir error cuando está en `remote` ejecución de la función de tabla cuando se usaron restricciones incorrectas para `getStructureOfRemoteTable`. [\#4009](https://github.com/ClickHouse/ClickHouse/pull/4009) ([alesapin](https://github.com/alesapin)) +- Corregir una fuga de enchufes netlink. Se colocaron en un grupo donde nunca se eliminaron y se crearon nuevos sockets al comienzo de un nuevo subproceso cuando todos los sockets actuales estaban en uso. [\#4017](https://github.com/ClickHouse/ClickHouse/pull/4017) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Corregir un error con el cierre `/proc/self/fd` directorio antes de todos los fds fueron leídos de `/proc` después de bifurcar `odbc-bridge` subproceso. [\#4120](https://github.com/ClickHouse/ClickHouse/pull/4120) ([alesapin](https://github.com/alesapin)) +- Conversión monótona de cadena a UInt fija en caso de uso de cadena en clave primaria. [\#3870](https://github.com/ClickHouse/ClickHouse/pull/3870) ([Invierno Zhang](https://github.com/zhang2014)) +- Se corrigió el error en el cálculo de la monotonía de la función de conversión de enteros. [\#3921](https://github.com/ClickHouse/ClickHouse/pull/3921) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Segfault fijo en `arrayEnumerateUniq`, `arrayEnumerateDense` funciones en caso de algunos argumentos no válidos. [\#3909](https://github.com/ClickHouse/ClickHouse/pull/3909) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparar UB en StorageMerge. [\#3910](https://github.com/ClickHouse/ClickHouse/pull/3910) ([Amos pájaro](https://github.com/amosbird)) +- Segfault fijo en funciones `addDays`, `subtractDays`. [\#3913](https://github.com/ClickHouse/ClickHouse/pull/3913) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregido el error: funciones `round`, `floor`, `trunc`, `ceil` puede devolver un resultado falso cuando se ejecuta en un argumento entero y una gran escala negativa. [\#3914](https://github.com/ClickHouse/ClickHouse/pull/3914) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió un error inducido por ‘kill query sync’ lo que conduce a un volcado central. [\#3916](https://github.com/ClickHouse/ClickHouse/pull/3916) ([muVulDeePecker](https://github.com/fancyqlx)) +- Corregir un error con un largo retraso después de la cola de replicación vacía. [\#3928](https://github.com/ClickHouse/ClickHouse/pull/3928) [\#3932](https://github.com/ClickHouse/ClickHouse/pull/3932) ([alesapin](https://github.com/alesapin)) +- Se corrigió el uso excesivo de memoria en caso de insertar en la tabla con `LowCardinality` clave primaria. [\#3955](https://github.com/ClickHouse/ClickHouse/pull/3955) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Fijo `LowCardinality` serialización para `Native` formato en caso de matrices vacías. [\#3907](https://github.com/ClickHouse/ClickHouse/issues/3907) [\#4011](https://github.com/ClickHouse/ClickHouse/pull/4011) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Se corrigió el resultado incorrecto al usar distinta por una sola columna numérica LowCardinality. [\#3895](https://github.com/ClickHouse/ClickHouse/issues/3895) [\#4012](https://github.com/ClickHouse/ClickHouse/pull/4012) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Se corrigió la agregación especializada con la clave LowCardinality (en caso de `compile` está habilitada la configuración). [\#3886](https://github.com/ClickHouse/ClickHouse/pull/3886) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Corregir el reenvío de usuarios y contraseñas para consultas de tablas replicadas. [\#3957](https://github.com/ClickHouse/ClickHouse/pull/3957) ([alesapin](https://github.com/alesapin)) ([小路](https://github.com/nicelulu)) +- Se corrigió una condición de carrera muy rara que puede ocurrir al enumerar tablas en la base de datos de diccionarios mientras recargaba diccionarios. [\#3970](https://github.com/ClickHouse/ClickHouse/pull/3970) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el resultado incorrecto cuando se usaba HAVING con ROLLUP o CUBE. [\#3756](https://github.com/ClickHouse/ClickHouse/issues/3756) [\#3837](https://github.com/ClickHouse/ClickHouse/pull/3837) ([Más información](https://github.com/reflection)) +- Se corrigieron alias de columna para consultas con `JOIN ON` sintaxis y tablas distribuidas. [\#3980](https://github.com/ClickHouse/ClickHouse/pull/3980) ([Invierno Zhang](https://github.com/zhang2014)) +- Se corrigió un error en la implementación interna de `quantileTDigest` (encontrado por Artem Vakhrushev). Este error nunca ocurre en ClickHouse y fue relevante solo para aquellos que usan la base de código ClickHouse como una biblioteca directamente. [\#3935](https://github.com/ClickHouse/ClickHouse/pull/3935) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Mejora {#improvements-6} + +- Soporte para `IF NOT EXISTS` en `ALTER TABLE ADD COLUMN` declaraciones junto con `IF EXISTS` en `DROP/MODIFY/CLEAR/COMMENT COLUMN`. [\#3900](https://github.com/ClickHouse/ClickHouse/pull/3900) ([Boris Granveaud](https://github.com/bgranvea)) +- Función `parseDateTimeBestEffort`: soporte para formatos `DD.MM.YYYY`, `DD.MM.YY`, `DD-MM-YYYY`, `DD-Mon-YYYY`, `DD/Month/YYYY` y similares. [\#3922](https://github.com/ClickHouse/ClickHouse/pull/3922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- `CapnProtoInputStream` ahora soportan estructuras dentadas. [\#4063](https://github.com/ClickHouse/ClickHouse/pull/4063) ([Odin Hultgren Van Der Horst](https://github.com/Miniwoffer)) +- Mejora de la usabilidad: se agregó una verificación de que el proceso del servidor se inicia desde el propietario del directorio de datos. No permita iniciar el servidor desde la raíz si los datos pertenecen a un usuario no root. [\#3785](https://github.com/ClickHouse/ClickHouse/pull/3785) ([Más información](https://github.com/sergey-v-galtsev)) +- Mejor lógica de verificación de columnas requeridas durante el análisis de consultas con JOINs. [\#3930](https://github.com/ClickHouse/ClickHouse/pull/3930) ([Artem Zuikov](https://github.com/4ertus2)) +- Disminución del número de conexiones en caso de un gran número de tablas distribuidas en un único servidor. [\#3726](https://github.com/ClickHouse/ClickHouse/pull/3726) ([Invierno Zhang](https://github.com/zhang2014)) +- Fila de totales admitidos para `WITH TOTALS` consulta para el controlador ODBC. [\#3836](https://github.com/ClickHouse/ClickHouse/pull/3836) ([Maksim Koritckiy](https://github.com/nightweb)) +- Permitido utilizar `Enum`s como enteros dentro de la función if. [\#3875](https://github.com/ClickHouse/ClickHouse/pull/3875) ([Ivan](https://github.com/abyss7)) +- Añadir `low_cardinality_allow_in_native_format` configuración. Si está desactivado, no utilice `LowCadrinality` escriba en `Native` formato. [\#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) ([KochetovNicolai](https://github.com/KochetovNicolai)) +- Se eliminaron algunos objetos redundantes de la caché de expresiones compiladas para reducir el uso de memoria. [\#4042](https://github.com/ClickHouse/ClickHouse/pull/4042) ([alesapin](https://github.com/alesapin)) +- Añadir comprobar que `SET send_logs_level = 'value'` consulta acepta el valor apropiado. [\#3873](https://github.com/ClickHouse/ClickHouse/pull/3873) ([Sabyanin Maxim](https://github.com/s-mx)) +- Verificación de tipo de datos fijos en las funciones de conversión de tipo. [\#3896](https://github.com/ClickHouse/ClickHouse/pull/3896) ([Invierno Zhang](https://github.com/zhang2014)) + +#### Mejoras de rendimiento {#performance-improvements-5} + +- Agregar una configuración MergeTree `use_minimalistic_part_header_in_zookeeper`. Si está habilitada, las tablas replicadas almacenarán metadatos de piezas compactas en un único znode de piezas. Esto puede reducir drásticamente el tamaño de la instantánea ZooKeeper (especialmente si las tablas tienen muchas columnas). Tenga en cuenta que después de habilitar esta configuración, no podrá degradar a una versión que no la admita. [\#3960](https://github.com/ClickHouse/ClickHouse/pull/3960) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Agregar una implementación basada en DFA para funciones `sequenceMatch` y `sequenceCount` en caso de que el patrón no contenga tiempo. [\#4004](https://github.com/ClickHouse/ClickHouse/pull/4004) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) +- Mejora del rendimiento para la serialización de números enteros. [\#3968](https://github.com/ClickHouse/ClickHouse/pull/3968) ([Amos pájaro](https://github.com/amosbird)) +- Cero a la izquierda relleno PODArray de modo que -1 elemento es siempre válido y puesto a cero. Se utiliza para el cálculo sin ramas de compensaciones. [\#3920](https://github.com/ClickHouse/ClickHouse/pull/3920) ([Amos pájaro](https://github.com/amosbird)) +- Revertir `jemalloc` versión que conducen a la degradación del rendimiento. [\#4018](https://github.com/ClickHouse/ClickHouse/pull/4018) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Cambios incompatibles hacia atrás {#backward-incompatible-changes-2} + +- Se eliminó la función no documentada `ALTER MODIFY PRIMARY KEY` porque fue reemplazado por el `ALTER MODIFY ORDER BY` comando. [\#3887](https://github.com/ClickHouse/ClickHouse/pull/3887) ([Método de codificación de datos:](https://github.com/ztlpn)) +- Función eliminada `shardByHash`. [\#3833](https://github.com/ClickHouse/ClickHouse/pull/3833) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Prohibir el uso de subconsultas escalares con el resultado del tipo `AggregateFunction`. [\#3865](https://github.com/ClickHouse/ClickHouse/pull/3865) ([Ivan](https://github.com/abyss7)) + +#### Mejoras de compilación / prueba / empaquetado {#buildtestingpackaging-improvements-6} + +- Añadido soporte para PowerPC (`ppc64le`) construir. [\#4132](https://github.com/ClickHouse/ClickHouse/pull/4132) ([Más información](https://github.com/danlark1)) +- Las pruebas funcionales con estado se ejecutan en el conjunto de datos público disponible. [\#3969](https://github.com/ClickHouse/ClickHouse/pull/3969) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el error cuando el servidor no puede comenzar con el `bash: /usr/bin/clickhouse-extract-from-config: Operation not permitted` mensaje dentro de Docker o systemd-nspawn. [\#4136](https://github.com/ClickHouse/ClickHouse/pull/4136) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar `rdkafka` Biblioteca a v1.0.0-RC5. Usado cppkafka en lugar de la interfaz C sin procesar. [\#4025](https://github.com/ClickHouse/ClickHouse/pull/4025) ([Ivan](https://github.com/abyss7)) +- Actualizar `mariadb-client` biblioteca. Se corrigió uno de los problemas encontrados por UBSan. [\#3924](https://github.com/ClickHouse/ClickHouse/pull/3924) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Algunas correcciones para compilaciones de UBSan. [\#3926](https://github.com/ClickHouse/ClickHouse/pull/3926) [\#3021](https://github.com/ClickHouse/ClickHouse/pull/3021) [\#3948](https://github.com/ClickHouse/ClickHouse/pull/3948) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron ejecuciones de pruebas por compromiso con la compilación de UBSan. +- Se agregaron ejecuciones por compromiso del analizador estático PVS-Studio. +- Corregidos errores encontrados por PVS-Studio. [\#4013](https://github.com/ClickHouse/ClickHouse/pull/4013) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigieron problemas de compatibilidad con glibc. [\#4100](https://github.com/ClickHouse/ClickHouse/pull/4100) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mueva las imágenes de Docker a 18.10 y agregue el archivo de compatibilidad para glibc\> = 2.28 [\#3965](https://github.com/ClickHouse/ClickHouse/pull/3965) ([alesapin](https://github.com/alesapin)) +- Agregue la variable env si el usuario no desea chown directorios en la imagen Docker del servidor. [\#3967](https://github.com/ClickHouse/ClickHouse/pull/3967) ([alesapin](https://github.com/alesapin)) +- Habilitado la mayoría de las advertencias de `-Weverything` en clang. Permitir `-Wpedantic`. [\#3986](https://github.com/ClickHouse/ClickHouse/pull/3986) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregaron algunas advertencias más que están disponibles solo en clang 8. [\#3993](https://github.com/ClickHouse/ClickHouse/pull/3993) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Enlace a `libLLVM` en lugar de a las librerías LLVM individuales cuando se usan enlaces compartidos. [\#3989](https://github.com/ClickHouse/ClickHouse/pull/3989) ([Películas De Sexo](https://github.com/orivej)) +- Se agregaron variables de desinfectante para imágenes de prueba. [\#4072](https://github.com/ClickHouse/ClickHouse/pull/4072) ([alesapin](https://github.com/alesapin)) +- `clickhouse-server` paquete debian recomendará `libcap2-bin` paquete a utilizar `setcap` herramienta para el establecimiento de capacidades. Esto es opcional. [\#4093](https://github.com/ClickHouse/ClickHouse/pull/4093) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejora del tiempo de compilación, fijo incluye. [\#3898](https://github.com/ClickHouse/ClickHouse/pull/3898) ([propulsor](https://github.com/proller)) +- Se agregaron pruebas de rendimiento para funciones hash. [\#3918](https://github.com/ClickHouse/ClickHouse/pull/3918) ([filimonov](https://github.com/filimonov)) +- Se corrigieron las dependencias de la biblioteca cíclica. [\#3958](https://github.com/ClickHouse/ClickHouse/pull/3958) ([propulsor](https://github.com/proller)) +- Compilación mejorada con poca memoria disponible. [\#4030](https://github.com/ClickHouse/ClickHouse/pull/4030) ([propulsor](https://github.com/proller)) +- Se agregó script de prueba para reproducir la degradación del rendimiento en `jemalloc`. [\#4036](https://github.com/ClickHouse/ClickHouse/pull/4036) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigieron errores ortográficos en comentarios y literales de cadena bajo `dbms`. [\#4122](https://github.com/ClickHouse/ClickHouse/pull/4122) ([maiha](https://github.com/maiha)) +- Se corrigieron errores tipográficos en los comentarios. [\#4089](https://github.com/ClickHouse/ClickHouse/pull/4089) ([Evgenii Pravda](https://github.com/kvinty)) diff --git a/docs/es/changelog/index.md b/docs/es/changelog/index.md deleted file mode 120000 index 79b747aee1b..00000000000 --- a/docs/es/changelog/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../CHANGELOG.md \ No newline at end of file diff --git a/docs/es/changelog/index.md b/docs/es/changelog/index.md new file mode 100644 index 00000000000..4ee7c076082 --- /dev/null +++ b/docs/es/changelog/index.md @@ -0,0 +1,627 @@ +--- +machine_translated: true +--- + +## Lanzamiento de ClickHouse v20.3 {#clickhouse-release-v20-3} + +### Todos los derechos reservados. {#clickhouse-release-v20-3-3-6-2020-03-17} + +### Corrección de errores {#bug-fix} + +- Añadir ajuste `use_compact_format_in_distributed_parts_names` que permite escribir archivos para `INSERT` consultas en `Distributed` mesa con un formato más compacto. Esto corrige [\#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [\#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). Hace que la versión 20.3 sea compatible con versiones anteriores de nuevo. +- Corregir un error en una replicación que no permite que la replicación funcione si el usuario ha ejecutado mutaciones en la versión anterior. Esto corrige [\#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [\#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). Hace que la versión 20.3 sea compatible con versiones anteriores de nuevo. +- Se corrigieron los nombres de funciones internas incorrectos para `sumKahan` y `sumWithOverflow`. Conduzco a una excepción mientras uso estas funciones en consultas remotas. [\#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). Este problema estaba en todas las versiones de ClickHouse. +- Se solucionó el problema: la zona horaria no se conservaba si escribía una expresión aritmética simple como `time + 1` (en contraste con una expresión como `time + INTERVAL 1 SECOND`). Esto corrige [\#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [\#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). Este problema estaba en todas las versiones de ClickHouse. +- Corregir posibles excepciones `Size of filter doesn't match size of column` y `Invalid number of rows in Chunk` en `MergeTreeRangeReader`. Podrían aparecer mientras se ejecuta `PREWHERE` en algunos casos. Fijar [\#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [\#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) +- Permitir `ALTER ON CLUSTER` de `Distributed` tablas con replicación interna. Esto corrige [\#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [\#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). Este problema estaba en todas las versiones de ClickHouse. + +### Todos los derechos reservados. {#clickhouse-release-v20-3-2-1-2020-03-12} + +### Cambio incompatible hacia atrás {#backward-incompatible-change} + +- Se ha solucionado el problema `file name too long` al enviar datos para `Distributed` para un gran número de réplicas. Se corrigió el problema de que las credenciales de réplica se expusieran en el registro del servidor. El formato del nombre del directorio en el disco se cambió a `[shard{shard_index}[_replica{replica_index}]]`. [\#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([Mikhail Korotov](https://github.com/millb)) Después de actualizar a la nueva versión, no podrá degradar sin intervención manual, porque la versión anterior del servidor no reconoce el nuevo formato de directorio. Si desea degradar, debe cambiar el nombre manualmente de los directorios correspondientes al formato anterior. Este cambio sólo es relevante si ha utilizado `INSERT`s a `Distributed` tabla. En la versión 20.3.3 introduciremos una configuración que le permitirá habilitar el nuevo formato gradualmente. +- Se ha cambiado el formato de las entradas de registro de replicación para los comandos de mutación. Tienes que esperar a que las mutaciones antiguas se procesen antes de instalar la nueva versión. +- Implemente un generador de perfiles de memoria simple que vuelca stacktraces a `system.trace_log` cada N bytes sobre el límite de asignación suave [\#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([Ivan](https://github.com/abyss7)) [\#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([alexey-milovidov](https://github.com/alexey-milovidov)) La columna de `system.trace_log` fue renombrado desde `timer_type` a `trace_type`. Esto requerirá cambios en el análisis de rendimiento de terceros y herramientas de procesamiento de flamegraph. +- Use la identificación de subproceso del sistema operativo en todas partes en lugar del número de subproceso interno. Esto corrige [\#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) Antiguo `clickhouse-client` no puede recibir registros que se envían desde el servidor cuando `send_logs_level` está habilitado, porque se han cambiado los nombres y tipos de los mensajes de registro estructurados. Por otro lado, diferentes versiones de servidor pueden enviar registros con diferentes tipos entre sí. Cuando usted no utiliza el `send_logs_level` ajuste, no debería importarle. [\#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Quitar `indexHint` función [\#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Quitar `findClusterIndex`, `findClusterValue` función. Esto corrige [\#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). Si estaba utilizando estas funciones, envíe un correo electrónico a `clickhouse-feedback@yandex-team.com` [\#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora no está permitido crear columnas o agregar columnas con `SELECT` subconsulta como expresión predeterminada. [\#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([alesapin](https://github.com/alesapin)) +- Requiere alias para subconsultas en JOIN. [\#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) +- Mejorar `ALTER MODIFY/ADD` consultas lógica. Ahora no puedes `ADD` sin tipo, `MODIFY` expresión predeterminada no cambia el tipo de columna y `MODIFY` type no pierde el valor de expresión predeterminado. Fijar [\#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [\#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([alesapin](https://github.com/alesapin)) +- Requiere que el servidor se reinicie para aplicar los cambios en la configuración de registro. Esta es una solución temporal para evitar el error en el que el servidor inicia sesión en un archivo de registro eliminado (consulte [\#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [\#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Configuración `experimental_use_processors` está habilitado de forma predeterminada. Esta configuración permite el uso de la nueva canalización de consultas. Esto es refactorización interna y no esperamos cambios visibles. Si ves algún problema, configúralo en cero. [\#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Novedad {#new-feature} + +- Añadir `Avro` y `AvroConfluent` Formatos de entrada/salida [\#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([Andrés Onyshchuk](https://github.com/oandrew)) [\#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([Andrés Onyshchuk](https://github.com/oandrew)) [\#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizaciones de subprocesos múltiples y sin bloqueo de claves caducadas en `cache` diccionarios (con permiso opcional para leer los antiguos). [\#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Agregar consulta `ALTER ... MATERIALIZE TTL`. Ejecuta la mutación que obliga a eliminar los datos caducados por TTL y recalcula la metainformación sobre TTL en todas las partes. [\#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([Anton Popov](https://github.com/CurtizJ)) +- Cambie de HashJoin a MergeJoin (en el disco) si es necesario [\#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `MOVE PARTITION` comando para `ALTER TABLE` [\#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [\#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Recarga de la configuración de almacenamiento desde el archivo de configuración sobre la marcha. [\#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Permitido cambiar `storage_policy` a uno no menos rico. [\#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se agregó soporte para globs / wildcards para el almacenamiento S3 y la función de mesa. [\#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Implementar `bitAnd`, `bitOr`, `bitXor`, `bitNot` para `FixedString(N)` Tipo de datos. [\#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Función añadida `bitCount`. Esto corrige [\#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [\#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([alexey-milovidov](https://github.com/alexey-milovidov)) [\#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([kopylov](https://github.com/ikopylov)) +- Añadir `generateRandom` función de tabla para generar filas aleatorias con un esquema dado. Permite rellenar la tabla de prueba arbitraria con datos. [\#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) +- `JSONEachRowFormat`: apoyar caso especial cuando los objetos encerrados en la matriz de nivel superior. [\#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([Kruglov Pavel](https://github.com/Avogar)) +- Ahora es posible crear una columna con `DEFAULT` expresión que depende de una columna con el valor predeterminado `ALIAS` expresion. [\#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([alesapin](https://github.com/alesapin)) +- Permitir especificar `--limit` más que el tamaño de los datos de origen en `clickhouse-obfuscator`. Los datos se repetirán con diferentes semillas aleatorias. [\#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `groupArraySample` función (similar a `groupArray`) con algoritmo de muestreo de reservorio. [\#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([Amos pájaro](https://github.com/amosbird)) +- Ahora puede controlar el tamaño de la cola de actualización en `cache`/`complex_key_cache` diccionarios a través de métricas del sistema. [\#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Permitir usar CRLF como separador de línea en formato de salida CSV con configuración `output_format_csv_crlf_end_of_line` se establece en 1 [\#8934](https://github.com/ClickHouse/ClickHouse/pull/8934) [\#8935](https://github.com/ClickHouse/ClickHouse/pull/8935) [\#8963](https://github.com/ClickHouse/ClickHouse/pull/8963) ([Mikhail Korotov](https://github.com/millb)) +- Implementar más funciones del [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` y `stringToH3` [\#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) +- Nueva configuración introducida: `max_parser_depth` para controlar el tamaño máximo de la pila y permitir grandes consultas complejas. Esto corrige [\#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) y [\#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [\#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([Maxim Smirnov](https://github.com/qMBQx8GH)) +- Añadir una configuración `force_optimize_skip_unused_shards` configuración para lanzar si no es posible omitir fragmentos no utilizados [\#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) +- Permitir configurar varios discos / volúmenes para almacenar datos para enviar `Distributed` motor [\#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) +- Política de almacenamiento de soporte (``) para almacenar datos temporales. [\#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) +- Añadir `X-ClickHouse-Exception-Code` Encabezado HTTP que se establece si se lanzó una excepción antes de enviar datos. Esto implementa [\#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [\#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([Mikhail Korotov](https://github.com/millb)) +- Función añadida `ifNotFinite`. Es solo un azúcar sintáctico: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [\#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `last_successful_update_time` columna en `system.dictionaries` tabla [\#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Añadir `blockSerializedSize` función (tamaño en disco sin compresión) [\#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) +- Añadir función `moduloOrZero` [\#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([Hcz](https://github.com/hczhcz)) +- Tablas de sistema añadidas `system.zeros` y `system.zeros_mt` así como funciones de cuento `zeros()` y `zeros_mt()`. Las tablas (y funciones de tabla) contienen una sola columna con nombre `zero` y tipo `UInt8`. Esta columna contiene ceros. Es necesario para fines de prueba como el método más rápido para generar muchas filas. Esto corrige [\#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [\#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +### Característica Experimental {#experimental-feature} + +- Añadir nuevo formato compacto de piezas en `MergeTree`-family tablas en las que todas las columnas se almacenan en un archivo. Esto ayuda a aumentar el rendimiento de las inserciones pequeñas y frecuentes. El formato antiguo (un archivo por columna) ahora se llama ancho. El formato de almacenamiento de datos se controla mediante la configuración `min_bytes_for_wide_part` y `min_rows_for_wide_part`. [\#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([Anton Popov](https://github.com/CurtizJ)) +- Soporte para almacenamiento S3 para `Log`, `TinyLog` y `StripeLog` tabla. [\#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([Pavel Kovalenko](https://github.com/Jokser)) + +### Corrección de errores {#bug-fix-1} + +- Se corrigieron espacios en blanco inconsistentes en los mensajes de registro. [\#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir un error en el que las matrices de tuplas sin nombre se aplanaban como estructuras anidadas en la creación de la tabla. [\#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) +- Se corrigió el problema cuando “Too many open files” puede ocurrir un error si hay demasiados archivos que coincidan con el patrón glob en `File` mesa o `file` función de la tabla. Ahora los archivos se abren perezosamente. Esto corrige [\#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [\#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- DROP TEMPORARY TABLE ahora solo deja caer la tabla temporal. [\#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([Vitaly Baranov](https://github.com/vitlibar)) +- Elimine la partición obsoleta cuando apagamos el servidor o DETACH / ATTACH una tabla. [\#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Para saber cómo el disco predeterminado calcula el espacio libre de `data` subdirectorio. Se ha solucionado el problema cuando la cantidad de espacio libre no se calcula correctamente si el `data` el directorio está montado en un dispositivo separado (caso raro). Esto corrige [\#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [\#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([Mikhail Korotov](https://github.com/millb)) +- Permitir coma (cruz) unirse con IN () dentro. [\#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) +- Permita reescribir CROSS a INNER JOIN si hay un operador \[NOT\] LIKE en la sección WHERE. [\#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) +- Corregir posible resultado incorrecto después `GROUP BY` con configuración habilitada `distributed_aggregation_memory_efficient`. Fijar [\#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [\#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Las claves encontradas se contaron como perdidas en las métricas de los diccionarios de caché. [\#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Corregir la incompatibilidad del protocolo de replicación introducida en [\#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [\#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) +- Condición de carrera fija en `queue_task_handle` en el inicio de `ReplicatedMergeTree` tabla. [\#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Token `NOT` no funcionó en `SHOW TABLES NOT LIKE` consulta [\#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [\#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Comprobación de rango añadida para funcionar `h3EdgeLengthM`. Sin esta comprobación, el desbordamiento del búfer es posible. [\#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió un error en los cálculos por lotes de OP lógicos ternarios en múltiples argumentos (más de 10). [\#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alejandro Kazakov](https://github.com/Akazz)) +- Corregir el error de optimización PREWHERE, que podría conducir a fallas según segfaults o `Inconsistent number of columns got from MergeTreeRangeReader` salvedad. [\#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([Anton Popov](https://github.com/CurtizJ)) +- Arreglar inesperado `Timeout exceeded while reading from socket` excepción, que ocurre aleatoriamente en la conexión segura antes de que se exceda el tiempo de espera y cuando se habilita el generador de perfiles de consultas. También añadir `connect_timeout_with_failover_secure_ms` configuración (por defecto 100 ms), que es similar a `connect_timeout_with_failover_ms`, pero se usa para conexiones seguras (porque el protocolo de enlace SSL es más lento que la conexión TCP ordinaria) [\#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([Tavplubix](https://github.com/tavplubix)) +- Corregir error con finalización de mutaciones, cuando la mutación puede colgarse en estado con `parts_to_do=0` y `is_done=0`. [\#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([alesapin](https://github.com/alesapin)) +- Use la nueva lógica ANY JOIN con `partial_merge_join` configuración. Es posible hacer `ANY|ALL|SEMI LEFT` y `ALL INNER` se une con `partial_merge_join=1` ahora. [\#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) +- Shard ahora sujeta la configuración obtenida del iniciador a los restos del fragmento en lugar de lanzar una excepción. Esta corrección permite enviar consultas a un fragmento con otras restricciones. [\#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([Vitaly Baranov](https://github.com/vitlibar)) +- Se corrigió el problema de administración de memoria en `MergeTreeReadPool`. [\#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fijar `toDecimal*OrNull()` familia de funciones cuando se llama con cadena `e`. Fijar [\#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [\#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) +- Asegúrese de que `FORMAT Null` no envía datos al cliente. [\#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Corregir error en la marca de tiempo `LiveViewBlockInputStream` no se actualizará. `LIVE VIEW` es una característica experimental. [\#8644](https://github.com/ClickHouse/ClickHouse/pull/8644) ([vxider](https://github.com/Vxider)) [\#8625](https://github.com/ClickHouse/ClickHouse/pull/8625) ([vxider](https://github.com/Vxider)) +- Fijo `ALTER MODIFY TTL` comportamiento incorrecto que no permitía eliminar expresiones TTL antiguas. [\#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Informe UBSan fijo en MergeTreeIndexSet. Esto corrige [\#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [\#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió el comportamiento de `match` y `extract` funciona cuando haystack tiene cero bytes. El comportamiento era incorrecto cuando el pajar era constante. Esto corrige [\#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [\#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [\#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite lanzar desde el destructor en la biblioteca Apache Avro de terceros. [\#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([Andrés Onyshchuk](https://github.com/oandrew)) +- No confirmar un lote encuestado desde `Kafka` parcialmente, ya que puede conducir a agujeros en los datos. [\#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) +- Fijar `joinGet` con tipos de devolución anulables. https://github.com/ClickHouse/ClickHouse/issues/8919 [\#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([Amos pájaro](https://github.com/amosbird)) +- Corregir la incompatibilidad de datos cuando se comprime con `T64` códec. [\#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2)) Fijar identificadores de tipo de datos en `T64` códec de compresión que conduce a una compresión incorrecta (de) en las versiones afectadas. [\#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir ajuste `enable_early_constant_folding` y desactivarlo en algunos casos que conduce a errores. [\#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) +- Repare el optimizador de predicados pushdown con VIEW y habilite la prueba [\#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([Invierno Zhang](https://github.com/zhang2014)) +- Arreglar segfault en `Merge` tablas, que pueden suceder al leer de `File` almacenamiento [\#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([Tavplubix](https://github.com/tavplubix)) +- Se agregó una verificación de la política de almacenamiento en `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. De lo contrario, podría hacer que los datos de la parte sean inaccesibles después del reinicio y evitar que se inicie ClickHouse. [\#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fix altera si hay TTL establecido para la tabla. [\#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir la condición de carrera que puede ocurrir cuando `SYSTEM RELOAD ALL DICTIONARIES` se ejecuta mientras se modifica / agrega / elimina algún diccionario. [\#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([Vitaly Baranov](https://github.com/vitlibar)) +- En versiones anteriores `Memory` el motor de base de datos utiliza una ruta de datos vacía, por lo que las tablas se crean en `path` directory (e.g. `/var/lib/clickhouse/`), not in data directory of database (e.g. `/var/lib/clickhouse/db_name`). [\#8753](https://github.com/ClickHouse/ClickHouse/pull/8753) ([Tavplubix](https://github.com/tavplubix)) +- Se corrigieron los mensajes de registro incorrectos sobre la falta de disco o política predeterminada. [\#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fix not(has()) para el índice bloom\_filter de los tipos de matriz. [\#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([chimbab](https://github.com/achimbab)) +- Permitir las primeras columnas en una tabla con `Log` motor ser un alias [\#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([Ivan](https://github.com/abyss7)) +- Corregir el orden de los rangos mientras se lee desde `MergeTree` mesa en un hilo. Podría dar lugar a excepciones de `MergeTreeRangeReader` o resultados de consultas incorrectos. [\#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([Anton Popov](https://github.com/CurtizJ)) +- Hacer `reinterpretAsFixedString` devolver `FixedString` en lugar de `String`. [\#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([Andrés Onyshchuk](https://github.com/oandrew)) +- Evite casos extremadamente raros cuando el usuario puede obtener un mensaje de error incorrecto (`Success` en lugar de una descripción detallada del error). [\#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- No se bloquee al usar `Template` formato con plantilla de fila vacía. [\#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Los archivos de metadatos para las tablas del sistema se pueden crear en un lugar incorrecto [\#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([Tavplubix](https://github.com/tavplubix)) Fijar [\#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). +- Corregir la carrera de datos en exception\_ptr en el diccionario de caché [\#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [\#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- No lanzar una excepción para la consulta `ATTACH TABLE IF NOT EXISTS`. Anteriormente se lanzaba si la tabla ya existe, a pesar de la `IF NOT EXISTS` clausula. [\#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([Anton Popov](https://github.com/CurtizJ)) +- Se corrigió la falta de cierre de paren en el mensaje de excepción. [\#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evitar el mensaje `Possible deadlock avoided` en el inicio de clickhouse-client en modo interactivo. [\#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se solucionó el problema cuando el relleno al final del valor codificado en base64 puede estar mal formado. Actualización de la biblioteca base64. Esto corrige [\#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), cerca [\#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [\#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evitar la pérdida de datos en `Kafka` en casos raros cuando la excepción ocurre después de leer el sufijo pero antes de confirmar. Fijar [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) +- Excepción fija en `DROP TABLE IF EXISTS` [\#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([Nikita Vasilev](https://github.com/nikvas0)) +- Corregir el bloqueo cuando un usuario intenta `ALTER MODIFY SETTING` para viejos `MergeTree` familia de motores de mesa. [\#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) +- Compatibilidad con números UInt64 que no caben en Int64 en funciones relacionadas con JSON. Actualizar SIMDJSON a maestro. Esto corrige [\#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [\#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la ejecución de predicados invertidos cuando se usa un índice funcional no estrictamente monotínico. [\#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([Alejandro Kazakov](https://github.com/Akazz)) +- No trate de doblar `IN` constante en `GROUP BY` [\#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([Amos pájaro](https://github.com/amosbird)) +- Corregir error en `ALTER DELETE` mutaciones que conduce a la corrupción del índice. Esto corrige [\#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) y [\#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). Además arreglar condiciones de carrera extremadamente raras en `ReplicatedMergeTree` `ALTER` consulta. [\#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([alesapin](https://github.com/alesapin)) +- Cuando el ajuste `compile_expressions` está habilitado, puede obtener `unexpected column` en `LLVMExecutableFunction` cuando usamos `Nullable` tipo [\#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Múltiples correcciones para `Kafka` engine: 1) corrige los duplicados que aparecían durante el reequilibrio del grupo de consumidores. 2) Arreglar raro ‘holes’ apareció cuando los datos se sondearon de varias particiones con una encuesta y se confirmaron parcialmente (ahora siempre procesamos / confirmamos todo el bloque de mensajes sondeados). 3) Arregle los lavados por tamaño de bloque (antes de eso, solo el lavado por tiempo de espera funcionaba correctamente). 4) mejor procedimiento de suscripción (con retroalimentación de la asignación). 5) Haga que las pruebas funcionen más rápido (con intervalos y tiempos de espera predeterminados). Debido al hecho de que los datos no se enjuagaron por el tamaño del bloque antes (como debería según la documentación), ese PR puede conducir a una cierta degradación del rendimiento con la configuración predeterminada (debido a los sofocos más a menudo y más pequeños que son menos óptimos). Si encuentra el problema de rendimiento después de ese cambio, aumente `kafka_max_block_size` en la tabla al valor más grande (por ejemplo `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). Fijar [\#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [\#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) +- Fijar `Parameter out of bound` excepción en algunas consultas después de las optimizaciones PREWHERE. [\#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Fallecimiento](https://github.com/bgiard)) +- Se corrigió el caso de la constancia mixta de los argumentos de la función `arrayZip`. [\#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Al ejecutar `CREATE` consulta, doblar expresiones constantes en argumentos del motor de almacenamiento. Reemplace el nombre de la base de datos vacía con la base de datos actual. Fijar [\#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [\#3492](https://github.com/ClickHouse/ClickHouse/issues/3492) [\#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) ([Tavplubix](https://github.com/tavplubix)) +- Ahora no es posible crear o agregar columnas con alias cíclicos simples como `a DEFAULT b, b DEFAULT a`. [\#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) +- Se ha corregido un error con doble movimiento que puede corromper parte original. Esto es relevante si usa `ALTER TABLE MOVE` [\#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Permitir `interval` identificador para analizar correctamente sin retrocesos. Solucionado el problema cuando una consulta no se puede ejecutar incluso si el `interval` identifier está encerrado en backticks o comillas dobles. Esto corrige [\#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [\#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Prueba de fuzz fija y comportamiento incorrecto de `bitTestAll`/`bitTestAny` función. [\#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir un posible bloqueo / número incorrecto de filas en `LIMIT n WITH TIES` cuando hay muchas filas iguales a n'th row. [\#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([Tavplubix](https://github.com/tavplubix)) +- Corregir mutaciones con partes escritas con habilitado `insert_quorum`. [\#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([alesapin](https://github.com/alesapin)) +- Corregir la carrera de datos en la destrucción de `Poco::HTTPServer`. Podría suceder cuando el servidor se inicia y se apaga inmediatamente. [\#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir un error en el que se mostraba un mensaje de error engañoso cuando se ejecutaba `SHOW CREATE TABLE a_table_that_does_not_exist`. [\#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) +- Fijo `Parameters are out of bound` excepción en algunos casos raros cuando tenemos una constante en el `SELECT` cláusula cuando tenemos una `ORDER BY` y una `LIMIT` clausula. [\#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Fijar mutaciones finalización, cuando ya hecho mutación puede tener estado `is_done=0`. [\#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([alesapin](https://github.com/alesapin)) +- Evitar la ejecución `ALTER ADD INDEX` para tablas MergeTree con sintaxis antigua, porque no funciona. [\#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([Mikhail Korotov](https://github.com/millb)) +- Durante el inicio del servidor, no acceda a la tabla, que `LIVE VIEW` depende de, por lo que el servidor podrá comenzar. También eliminar `LIVE VIEW` dependencias al separar `LIVE VIEW`. `LIVE VIEW` es una característica experimental. [\#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([Tavplubix](https://github.com/tavplubix)) +- Arreglar posible segfault en `MergeTreeRangeReader`, mientras se ejecuta `PREWHERE`. [\#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([Anton Popov](https://github.com/CurtizJ)) +- Corregir posibles sumas de comprobación no coincidentes con TTL de columna. [\#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) +- Se corrigió un error cuando las partes no se movían en segundo plano por las reglas TTL en caso de que solo haya un volumen. [\#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se ha solucionado el problema `Method createColumn() is not implemented for data type Set`. Esto corrige [\#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [\#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora intentaremos finalizar las mutaciones con más frecuencia. [\#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) +- Fijar `intDiv` por menos una constante [\#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([Hcz](https://github.com/hczhcz)) +- Corregir la posible condición de carrera en `BlockIO`. [\#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir un error que conduce a la terminación del servidor al intentar usar / soltar `Kafka` tabla creada con parámetros incorrectos. [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) +- Solución alternativa agregada si el sistema operativo devuelve un resultado incorrecto para `timer_create` función. [\#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregido el error en el uso de `min_marks_for_seek` parámetro. Se corrigió el mensaje de error cuando no hay una clave de fragmentación en la tabla distribuida e intentamos omitir fragmentos no utilizados. [\#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) + +### Mejora {#improvement} + +- Implementar `ALTER MODIFY/DROP` encima de mutaciones para `ReplicatedMergeTree*` familia de motores. Ahora `ALTERS` bloquea solo en la etapa de actualización de metadatos, y no bloquea después de eso. [\#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([alesapin](https://github.com/alesapin)) +- Agregue la capacidad de reescribir CROSS a INNER JOINs con `WHERE` sección que contiene nombres unqialified. [\#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) +- Hacer `SHOW TABLES` y `SHOW DATABASES` las consultas apoyan el `WHERE` expresiones y `FROM`/`IN` [\#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([Sundyli](https://github.com/sundy-li)) +- Añadido un ajuste `deduplicate_blocks_in_dependent_materialized_views`. [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) +- Después de los cambios recientes, el cliente MySQL comenzó a imprimir cadenas binarias en hexadecimal, lo que las hizo no legibles ([\#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). La solución en ClickHouse es marcar las columnas de cadena como UTF-8, lo que no siempre es, pero generalmente el caso. [\#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([Yuriy Baranov](https://github.com/yurriy)) +- Agregue soporte de las claves String y FixedString para `sumMap` [\#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Fallecimiento](https://github.com/bgiard)) +- Teclas de cadena de soporte en los mapas SummingMergeTree [\#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Fallecimiento](https://github.com/bgiard)) +- Terminación de la señal del subproceso al grupo de subprocesos incluso si el subproceso ha lanzado una excepción [\#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([Más información](https://github.com/dingxiangfei2009)) +- Permitir establecer `query_id` en `clickhouse-benchmark` [\#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([Anton Popov](https://github.com/CurtizJ)) +- No permita expresiones extrañas en `ALTER TABLE ... PARTITION partition` consulta. Esto aborda [\#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [\#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Tabla `system.table_engines` ahora proporciona información sobre el soporte de características (como `supports_ttl` o `supports_sort_order`). [\#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) +- Permitir `system.metric_log` predeterminada. Contendrá filas con valores de ProfileEvents, CurrentMetrics recopilados con “collect\_interval\_milliseconds” intervalo (un segundo por defecto). La tabla es muy pequeña (generalmente en orden de megabytes) y la recopilación de estos datos por defecto es razonable. [\#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries. Fixes [\#6964](https://github.com/ClickHouse/ClickHouse/issues/6964) [\#8874](https://github.com/ClickHouse/ClickHouse/pull/8874) ([Ivan](https://github.com/abyss7)) +- Ahora temporal `LIVE VIEW` es creado por `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` en lugar de `CREATE TEMPORARY LIVE VIEW ...`, porque la sintaxis anterior no era consistente con `CREATE TEMPORARY TABLE ...` [\#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([Tavplubix](https://github.com/tavplubix)) +- Agregar text\_log.parámetro de configuración de nivel para limitar las entradas `system.text_log` tabla [\#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) +- Permitir colocar la parte descargada en discos / volúmenes de acuerdo con las reglas TTL [\#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Para diccionarios MySQL externos, permita mutualizar el grupo de conexiones MySQL para “share” entre los diccionarios. Esta opción reduce significativamente el número de conexiones a los servidores MySQL. [\#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) +- Mostrar el tiempo de ejecución de consultas más cercano para los cuantiles en `clickhouse-benchmark` salida en lugar de valores interpolados. Es mejor mostrar valores que correspondan al tiempo de ejecución de algunas consultas. [\#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Posibilidad de agregar clave y marca de tiempo para el mensaje al insertar datos en Kafka. Fijar [\#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [\#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) +- Si el servidor se ejecuta desde el terminal, resalte el número de hilo, el id de consulta y la prioridad de registro por colores. Esto es para mejorar la legibilidad de los mensajes de registro correlacionados para los desarrolladores. [\#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor mensaje de excepción al cargar tablas para `Ordinary` base. [\#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Implementar `arraySlice` para matrices con estados de función agregados. Esto corrige [\#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [\#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permita que las funciones constantes y las matrices constantes se utilicen en el lado derecho del operador IN. [\#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([Anton Popov](https://github.com/CurtizJ)) +- Si la excepción del zookeeper ha ocurrido al obtener datos para el sistema.réplicas, mostrarlo en una columna separada. Esto implementa [\#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [\#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Elimine atómicamente las partes de datos de MergeTree en destroy. [\#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Soporte de seguridad a nivel de fila para tablas distribuidas. [\#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([Ivan](https://github.com/abyss7)) +- Now we recognize suffix (like KB, KiB…) in settings values. [\#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([Mikhail Korotov](https://github.com/millb)) +- Evitar la falta de memoria mientras se construye el resultado de una gran UNIÓN. [\#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) +- Se agregaron nombres de clústeres a sugerencias en modo interactivo en `clickhouse-client`. [\#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries [\#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([Ivan](https://github.com/abyss7)) +- Columna añadida `exception_code` en `system.query_log` tabla. [\#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([Mikhail Korotov](https://github.com/millb)) +- Servidor de compatibilidad MySQL habilitado en el puerto `9004` en el archivo de configuración del servidor predeterminado. Se corrigió el comando de generación de contraseñas en el ejemplo de configuración. [\#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([Yuriy Baranov](https://github.com/yurriy)) +- Evite abortar al apagar si el sistema de archivos es de solo lectura. Esto corrige [\#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [\#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor mensaje de excepción cuando se requiere longitud en la consulta HTTP POST. [\#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `_path` y `_file` columnas virtuales para `HDFS` y `File` motores y `hdfs` y `file` funciones de la tabla [\#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) +- Corregir error `Cannot find column` mientras se inserta en `MATERIALIZED VIEW` en caso de que se agregara una nueva columna a la tabla interna de la vista. [\#8766](https://github.com/ClickHouse/ClickHouse/pull/8766) [\#8788](https://github.com/ClickHouse/ClickHouse/pull/8788) ([vzakaznikov](https://github.com/vzakaznikov)) [\#8788](https://github.com/ClickHouse/ClickHouse/issues/8788) [\#8806](https://github.com/ClickHouse/ClickHouse/pull/8806) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [\#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corrige el progreso sobre el protocolo cliente-servidor nativo, al enviar el progreso después de la actualización final (como los registros). Esto puede ser relevante solo para algunas herramientas de terceros que usan protocolo nativo. [\#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) +- Agregue una métrica del sistema que rastree el número de conexiones de cliente utilizando el protocolo MySQL ([\#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [\#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([Eugene Klimov](https://github.com/Slach)) +- A partir de ahora, las respuestas HTTP tendrán `X-ClickHouse-Timezone` encabezado establecido en el mismo valor de zona horaria que `SELECT timezone()` informaría. [\#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) + +### Mejora del rendimiento {#performance-improvement} + +- Mejorar el rendimiento del índice de análisis con IN [\#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([Anton Popov](https://github.com/CurtizJ)) +- Código más simple y eficiente en funciones lógicas + limpiezas de código. Un seguimiento de [\#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [\#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([Alejandro Kazakov](https://github.com/Akazz)) +- Mejora general del rendimiento (en el rango del 5%..200% para consultas afectadas) garantizando aliasing aún más estricto con las características de C ++ 20. [\#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([Amos pájaro](https://github.com/amosbird)) +- Aliasing más estricto para bucles internos de funciones de comparación. [\#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Aliasing más estricto para bucles internos de funciones aritméticas. [\#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Una implementación ~ 3 veces más rápida para ColumnVector::replicate() , a través de la cual se implementa ColumnConst::convertToFullColumn() . También será útil en pruebas al materializar constantes. [\#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([Alejandro Kazakov](https://github.com/Akazz)) +- Otra mejora de rendimiento menor a `ColumnVector::replicate()` (esto acelera el `materialize` función y funciones de orden superior) una mejora aún más a [\#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [\#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([Alejandro Kazakov](https://github.com/Akazz)) +- Rendimiento mejorado de `stochasticLinearRegression` función de agregado. Este parche es aportado por Intel. [\#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejorar el rendimiento de `reinterpretAsFixedString` función. [\#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- No envíe bloques al cliente para `Null` formato en la tubería de procesadores. [\#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [\#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alejandro Kuzmenkov](https://github.com/akuzm)) + +### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement} + +- El manejo de excepciones ahora funciona correctamente en el subsistema Windows para Linux. Consulte https://github.com/ClickHouse-Extras/libunwind/pull/3 Esto corrige [\#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [\#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) +- Reemplazar `readline` con `replxx` para la edición de línea interactiva en `clickhouse-client` [\#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([Ivan](https://github.com/abyss7)) +- Mejor tiempo de compilación y menos instancias de plantillas en FunctionsComparison. [\#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Integración añadida con `clang-tidy` en CI. Ver también [\#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [\#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora vinculamos ClickHouse en CI usando `lld` incluso para `gcc`. [\#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([alesapin](https://github.com/alesapin)) +- Permitir aleatorizar la programación de subprocesos e insertar fallas cuando `THREAD_FUZZER_*` se establecen variables de entorno. Esto ayuda a las pruebas. [\#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Habilitar sockets seguros en pruebas sin estado [\#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([Tavplubix](https://github.com/tavplubix)) +- Hacer SPLIT\_SHARED\_LIBRARIES=OFF más robusto [\#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) +- Hacer “performance\_introspection\_and\_logging” prueba confiable al servidor aleatorio atascado. Esto puede suceder en el entorno de CI. Ver también [\#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [\#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Validar XML en la comprobación de estilo. [\#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Condición de carrera fija en la prueba `00738_lock_for_inner_table`. Esta prueba se basó en el sueño. [\#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Eliminar pruebas de rendimiento de tipo `once`. Esto es necesario para ejecutar todas las pruebas de rendimiento en modo de comparación estadística (más confiable). [\#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Prueba de rendimiento agregada para funciones aritméticas. [\#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Prueba de rendimiento añadida para `sumMap` y `sumMapWithOverflow` funciones agregadas. Seguimiento de [\#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [\#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Asegúrese de estilo de ErrorCodes por comprobación de estilo. [\#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregar script para el historial de pruebas. [\#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([alesapin](https://github.com/alesapin)) +- Añadir advertencia GCC `-Wsuggest-override` para localizar y arreglar todos los lugares donde `override` palabra clave debe ser utilizado. [\#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([Método de codificación de datos:](https://github.com/kreuzerkrieg)) +- Ignore el símbolo débil en Mac OS X porque debe definirse [\#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([Usuario eliminado](https://github.com/ghost)) +- Normalice el tiempo de ejecución de algunas consultas en las pruebas de rendimiento. Esto se hace en preparación para ejecutar todas las pruebas de rendimiento en modo de comparación. [\#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Arregle algunas pruebas para admitir pytest con pruebas de consulta [\#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([Ivan](https://github.com/abyss7)) +- Habilite SSL en compilación con MSan, por lo que el servidor no fallará al iniciarse al ejecutar pruebas sin estado [\#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([Tavplubix](https://github.com/tavplubix)) +- Corregir la sustitución de la base de datos en los resultados de las pruebas [\#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) +- Crear correcciones para plataformas diversas [\#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([propulsor](https://github.com/proller)) [\#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([propulsor](https://github.com/proller)) [\#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([propulsor](https://github.com/proller)) +- Sección de discos agregada a la imagen de la ventana acoplable de prueba sin estado con cobertura [\#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([Pavel Kovalenko](https://github.com/Jokser)) +- Deshágase de los archivos en el árbol de origen al compilar con GRPC [\#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([Amos pájaro](https://github.com/amosbird)) +- Un tiempo de compilación ligeramente más rápido al eliminar SessionCleaner del contexto. Haga que el código de SessionCleaner sea más simple. [\#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Comprobación actualizada de consultas colgadas en el script de prueba de clickhouse [\#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alejandro Kazakov](https://github.com/Akazz)) +- Se eliminaron algunos archivos inútiles del repositorio. [\#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Tipo cambiado de perftests matemáticos de `once` a `loop`. [\#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Agregue una imagen acoplable que permite construir un informe HTML del navegador de código interactivo para nuestra base de código. [\#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) Ver [Navegador de código Woboq](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/src/index.html) +- Suprima algunas fallas de prueba bajo MSan. [\#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Aceleración “exception while insert” prueba. Esta prueba a menudo se agota en la compilación de depuración con cobertura. [\#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar `libcxx` y `libcxxabi` dominar. En preparación para [\#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [\#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Reparar la prueba de flacky `00910_zookeeper_test_alter_compression_codecs`. [\#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Limpie las banderas duplicadas del vinculador. Asegúrese de que el enlazador no busque un símbolo inesperado. [\#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos pájaro](https://github.com/amosbird)) +- Añadir `clickhouse-odbc` conductor en imágenes de prueba. Esto permite probar la interacción de ClickHouse con ClickHouse a través de su propio controlador ODBC. [\#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) +- Corregir varios errores en las pruebas unitarias. [\#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([alesapin](https://github.com/alesapin)) +- Permitir `-Wmissing-include-dirs` Advertencia de GCC para eliminar todas las incluidas no existentes, principalmente como resultado de errores de scripting de CMake [\#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([Método de codificación de datos:](https://github.com/kreuzerkrieg)) +- Describa las razones si el generador de perfiles de consultas no puede funcionar. Esto está destinado a [\#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [\#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualice OpenSSL al maestro ascendente. Se ha solucionado el problema cuando las conexiones TLS pueden fallar con el mensaje `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` y `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. El problema estaba presente en la versión 20.1. [\#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar Dockerfile para el servidor [\#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) +- Correcciones menores en el script build-gcc-from-sources [\#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) +- Reemplazar `numbers` a `zeros` en perftests donde `number` no se utiliza la columna. Esto conducirá a resultados de pruebas más limpios. [\#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Solucione el problema de desbordamiento de pila al usar initializer\_list en los constructores de columnas. [\#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Usuario eliminado](https://github.com/ghost)) +- Actualizar librdkafka a v1.3.0. Habilitar incluido `rdkafka` y `gsasl` bibliotecas en Mac OS X. [\#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([Andrés Onyshchuk](https://github.com/oandrew)) +- corrección de compilación en GCC 9.2.0 [\#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) + +## Lanzamiento de ClickHouse v20.1 {#clickhouse-release-v20-1} + +### Todos los derechos reservados. {#clickhouse-release-v20-1-6-30-2020-03-05} + +#### Corrección de errores {#bug-fix-2} + +- Corregir la incompatibilidad de datos cuando se comprime con `T64` códec. + [\#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) +- Corregir el orden de los rangos mientras se lee de la tabla MergeTree en un hilo. Fijar [\#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). + [\#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ)](https://github.com/CurtizJ) +- Arreglar posible segfault en `MergeTreeRangeReader`, mientras se ejecuta `PREWHERE`. Fijar [\#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). + [\#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ)](https://github.com/CurtizJ) +- Fijar `reinterpretAsFixedString` devolver `FixedString` en lugar de `String`. + [\#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) +- Fijar `joinGet` con tipos de devolución anulables. Fijar [\#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) + [\#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) +- Repare la prueba de fuzz y el comportamiento incorrecto de las funciones bitTestAll / bitTestAny. + [\#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(alexey-milovidov)](https://github.com/alexey-milovidov) +- Corrige el comportamiento de las funciones de coincidencia y extracción cuando haystack tiene cero bytes. El comportamiento era incorrecto cuando el pajar era constante. Fijar [\#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) + [\#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(alexey-milovidov)](https://github.com/alexey-milovidov) +- Se corrigió la ejecución de predicados invertidos cuando se usa un índice funcional no estrictamente monotínico. Fijar [\#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) + [\#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz))](https://github.com/Akazz) +- Permitir reescribir `CROSS` a `INNER JOIN` si hay `[NOT] LIKE` operador en `WHERE` apartado. Fijar [\#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) + [\#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) +- Permitir que las primeras columnas de una tabla con Log engine sean un alias. + [\#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) +- Permitir la combinación de coma con `IN()` dentro. Fijar [\#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). + [\#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) +- Mejorar `ALTER MODIFY/ADD` consultas lógica. Ahora no puedes `ADD` sin tipo, `MODIFY` expresión predeterminada no cambia el tipo de columna y `MODIFY` type no pierde el valor de expresión predeterminado. Fijar [\#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). + [\#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) +- Fijar la finalización de las mutaciones, cuando la mutación ya hecha puede tener el estado is\_done = 0. + [\#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) +- Apoyo “Processors” tubería para el sistema.números y sistema.numbers\_mt. Esto también corrige el error cuando `max_execution_time` no se respeta. + [\#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) +- Corregir el conteo incorrecto de `DictCacheKeysRequestedFound` métrica. + [\#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) +- Se agregó una verificación de la política de almacenamiento en `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE` que de lo contrario podría hacer que los datos de la parte sean inaccesibles después del reinicio y evitar que se inicie ClickHouse. + [\#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) +- Informe UBSan fijo en `MergeTreeIndexSet`. Esto corrige [\#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) + [\#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(alexey-milovidov)](https://github.com/alexey-milovidov) +- Corrige un posible registro de datos en BlockIO. + [\#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) +- Soporte para `UInt64` números que no caben en Int64 en funciones relacionadas con JSON. Actualizar `SIMDJSON` dominar. Esto corrige [\#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) + [\#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(alexey-milovidov)](https://github.com/alexey-milovidov) +- Solucione el problema cuando la cantidad de espacio libre no se calcula correctamente si el directorio de datos está montado en un dispositivo separado. Para el disco predeterminado, calcule el espacio libre del subdirectorio de datos. Esto corrige [\#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) + [\#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(millb)](https://github.com/millb) +- Solucione el problema cuando las conexiones TLS pueden fallar con el mensaje `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error and SSL Exception: error:2400006E:random number generator::error retrieving entropy.` Actualice OpenSSL al maestro ascendente. + [\#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(alexey-milovidov)](https://github.com/alexey-milovidov) +- Al ejecutar `CREATE` consulta, doblar expresiones constantes en argumentos del motor de almacenamiento. Reemplace el nombre de la base de datos vacía con la base de datos actual. Fijar [\#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [\#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). También corrija la comprobación de la dirección local en ClickHouseDictionarySource. + [\#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) +- Arreglar segfault en `StorageMerge`, que puede suceder al leer de StorageFile. + [\#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) +- Evitar la pérdida de datos en `Kafka` en casos raros cuando la excepción ocurre después de leer el sufijo pero antes de confirmar. Fijar [\#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Relacionados: [\#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) + [\#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) +- Corregir un error que conduce a la terminación del servidor al intentar usar / soltar `Kafka` tabla creada con parámetros incorrectos. Fijar [\#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporar [\#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). + [\#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) + +#### Novedad {#new-feature-1} + +- Añadir `deduplicate_blocks_in_dependent_materialized_views` opción para controlar el comportamiento de las inserciones idempotentes en tablas con vistas materializadas. Esta nueva característica se agregó a la versión de corrección de errores mediante una solicitud especial de Altinity. + [\#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) + +### Todos los derechos reservados. {#clickhouse-release-v20-1-2-4-2020-01-22} + +### Cambio incompatible hacia atrás {#backward-incompatible-change-1} + +- Haga el ajuste `merge_tree_uniform_read_distribution` obsoleto. El servidor aún reconoce esta configuración, pero no tiene ningún efecto. [\#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Tipo de retorno cambiado de la función `greatCircleDistance` a `Float32` porque ahora el resultado del cálculo es `Float32`. [\#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora se espera que los parámetros de consulta estén representados en “escaped” formato. Por ejemplo, para pasar una cadena `ab` tienes que escribir `a\tb` o `a\b` y respectivamente, `a%5Ctb` o `a%5C%09b` en URL. Esto es necesario para agregar la posibilidad de pasar NULL como `\N`. Esto corrige [\#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [\#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Permitir `use_minimalistic_part_header_in_zookeeper` establecimiento de `ReplicatedMergeTree` predeterminada. Esto reducirá significativamente la cantidad de datos almacenados en ZooKeeper. Esta configuración es compatible desde la versión 19.1 y ya la usamos en producción en múltiples servicios sin problemas durante más de medio año. Deshabilite esta configuración si tiene la posibilidad de cambiar a versiones anteriores a 19.1. [\#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Los índices de omisión de datos están listos para producción y están habilitados de forma predeterminada. Configuración `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` y `allow_experimental_multiple_joins_emulation` ahora están obsoletos y no hacen nada. [\#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir nuevo `ANY JOIN` lógica para `StorageJoin` consistente con `JOIN` operación. Para actualizar sin cambios en el comportamiento, necesita agregar `SETTINGS any_join_distinct_right_table_keys = 1` a Engine Unir metadatos de tablas o volver a crear estas tablas después de la actualización. [\#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) +- Requiere que el servidor se reinicie para aplicar los cambios en la configuración de registro. Esta es una solución temporal para evitar el error en el que el servidor inicia sesión en un archivo de registro eliminado (consulte [\#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [\#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alejandro Kuzmenkov](https://github.com/akuzm)) + +### Novedad {#new-feature-2} + +- Se agregó información sobre las rutas de acceso de piezas a `system.merges`. [\#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Añadir capacidad de ejecutar `SYSTEM RELOAD DICTIONARY` consulta en `ON CLUSTER` modo. [\#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([Guillaume Tassery](https://github.com/YiuRULE)) +- Añadir capacidad de ejecutar `CREATE DICTIONARY` consultas en `ON CLUSTER` modo. [\#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([alesapin](https://github.com/alesapin)) +- Ahora el perfil del usuario en `users.xml` puede heredar varios perfiles. [\#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +- Añadir `system.stack_trace` tabla que permite ver los rastros de pila de todos los hilos del servidor. Esto es útil para los desarrolladores para examinar el estado del servidor. Esto corrige [\#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [\#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `DateTime64` tipo de datos con una precisión de subsegundo configurable. [\#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([Vasily Nemkov](https://github.com/Enmk)) +- Agregar función de tabla `clusterAllReplicas` que permite consultar todos los nodos en el clúster. [\#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([mujer madura](https://github.com/kiransunkari)) +- Agregar función agregada `categoricalInformationValue` que calcula el valor de información de una entidad discreta. [\#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([Hcz](https://github.com/hczhcz)) +- Acelerar el análisis de archivos de datos en `CSV`, `TSV` y `JSONEachRow` formato haciéndolo en paralelo. [\#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Añadir función `bankerRound` que realiza el redondeo del banquero. [\#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([Hcz](https://github.com/hczhcz)) +- Soporta más idiomas en el diccionario incrustado para nombres de región: ‘ru’, ‘en’, ‘ua’, ‘uk’, ‘by’, ‘kz’, ‘tr’, ‘de’, ‘uz’, ‘lv’, ‘lt’, ‘et’, ‘pt’, ‘he’, ‘vi’. [\#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejoras en la consistencia de `ANY JOIN` lógica. Ahora `t1 ANY LEFT JOIN t2` igual `t2 ANY RIGHT JOIN t1`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir ajuste `any_join_distinct_right_table_keys` que permite un comportamiento antiguo para `ANY INNER JOIN`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir nuevo `SEMI` y `ANTI JOIN`. Antiguo `ANY INNER JOIN` comportamiento ahora disponible como `SEMI LEFT JOIN`. [\#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `Distributed` formato para `File` motor y `file` función de mesa que permite leer desde `.bin` archivos generados por inserciones asincrónicas en `Distributed` tabla. [\#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Agregar argumento de columna de reinicio opcional para `runningAccumulate` que permite restablecer los resultados de agregación para cada nuevo valor clave. [\#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([Sergey Kononenko](https://github.com/kononencheg)) +- Agregue la capacidad de usar ClickHouse como punto final Prometheus. [\#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) +- Añadir sección `` en `config.xml` que restringe los hosts permitidos para motores de tabla remotos y funciones de tabla `URL`, `S3`, `HDFS`. [\#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([Mikhail Korotov](https://github.com/millb)) +- Función añadida `greatCircleAngle` que calcula la distancia en una esfera en grados. [\#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se ha cambiado el radio de la Tierra para que sea consistente con la biblioteca H3. [\#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `JSONCompactEachRow` y `JSONCompactEachRowWithNamesAndTypes` Formatos de entrada y salida. [\#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([Mikhail Korotov](https://github.com/millb)) +- Característica agregada para motores de tablas relacionados con archivos y funciones de tabla (`File`, `S3`, `URL`, `HDFS`) que permite leer y escribir `gzip` archivos basados en el parámetro del motor adicional o extensión de archivo. [\#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([Andrey Bodrov](https://github.com/apbodrov)) +- Se agregó el `randomASCII(length)` función, generando una cadena con un conjunto aleatorio de [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) caracteres imprimibles. [\#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([Bayoneta](https://github.com/BayoNet)) +- Función añadida `JSONExtractArrayRaw` que devuelve una matriz en elementos de matriz json no analizados de `JSON` cadena. [\#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) +- Añadir `arrayZip` función que permite combinar múltiples matrices de longitudes iguales en una matriz de tuplas. [\#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([Invierno Zhang](https://github.com/zhang2014)) +- Agregue la capacidad de mover datos entre discos de acuerdo con la configuración `TTL`-expresiones para `*MergeTree` familia de motores de mesa. [\#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se agregó una nueva función de agregado `avgWeighted` que permite calcular el promedio ponderado. [\#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([Andrey Bodrov](https://github.com/apbodrov)) +- Ahora el análisis paralelo está habilitado de forma predeterminada para `TSV`, `TSKV`, `CSV` y `JSONEachRow` formato. [\#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Añadir varias funciones geográficas de `H3` biblioteca: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` y `h3kRing`. [\#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) +- Añadido soporte para brotli (`br`) compresión en almacenes relacionados con archivos y funciones de tabla. Esto corrige [\#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [\#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Añadir `groupBit*` funciones para el `SimpleAggregationFunction` tipo. [\#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([Guillaume Tassery](https://github.com/YiuRULE)) + +### Corrección de errores {#bug-fix-3} + +- Corregir el cambio de nombre de las tablas con `Distributed` motor. Soluciona el problema [\#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [\#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([Tavplubix](https://github.com/tavplubix)) +- Ahora diccionarios de apoyo `EXPRESSION` para atributos en cadena arbitraria en dialecto SQL no ClickHouse. [\#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) +- Arreglar roto `INSERT SELECT FROM mysql(...)` consulta. Esto corrige [\#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) y [\#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [\#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([Tavplubix](https://github.com/tavplubix)) +- Corregir error “Mismatch column sizes” al insertar el valor predeterminado `Tuple` de `JSONEachRow`. Esto corrige [\#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [\#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([Tavplubix](https://github.com/tavplubix)) +- Ahora se lanzará una excepción en caso de usar `WITH TIES` junto `LIMIT BY`. También agregue la capacidad de usar `TOP` con `LIMIT BY`. Esto corrige [\#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [\#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Repara la dependencia del superintendente de la nueva versión de glibc en `clickhouse-odbc-bridge` binario. [\#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([Amos pájaro](https://github.com/amosbird)) +- Corregir error en la función de verificación de `*MergeTree` familia de motores. Ahora no falla en caso de que tengamos la misma cantidad de filas en el último gránulo y la última marca (no final). [\#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([alesapin](https://github.com/alesapin)) +- Fijar el inserto en `Enum*` columnas después `ALTER` consulta, cuando el tipo numérico subyacente es igual al tipo especificado en la tabla. Esto corrige [\#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [\#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([Anton Popov](https://github.com/CurtizJ)) +- Negativo no constante permitido “size” argumento para la función `substring`. No fue permitido por error. Esto corrige [\#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [\#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir el error de análisis cuando se pasa un número incorrecto de argumentos a `(O|J)DBC` motor de mesa. [\#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([alesapin](https://github.com/alesapin)) +- Usar el nombre de comando del proceso de clickhouse en ejecución al enviar registros a syslog. En versiones anteriores, se usaba una cadena vacía en lugar del nombre del comando. [\#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) +- Corregir la comprobación de hosts permitidos para `localhost`. Este PR corrige la solución proporcionada en [\#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [\#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) +- Arreglar accidente raro en `argMin` y `argMax` funciones para argumentos de cadena larga, cuando el resultado se usa en `runningAccumulate` función. Esto corrige [\#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [\#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([dinosaurio](https://github.com/769344359)) +- Repara el sobrecompromiso de memoria para tablas con `Buffer` motor. [\#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) +- Se corrigió un error potencial en las funciones que pueden tomar `NULL` como uno de los argumentos y devuelve no NULL. [\#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejores cálculos de métricas en el grupo de subprocesos para procesos en segundo plano para `MergeTree` motores de mesa. [\#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fijar la función `IN` dentro `WHERE` declaración cuando el filtro de tabla de nivel de fila está presente. Fijar [\#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [\#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([Ivan](https://github.com/abyss7)) +- Ahora se lanza una excepción si el valor integral no se analiza completamente para los valores de configuración. [\#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([Mikhail Korotov](https://github.com/millb)) +- Solucionar la excepción cuando se utiliza la función de agregado en la consulta a la tabla distribuida con más de dos fragmentos locales. [\#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) +- Ahora el filtro bloom puede manejar matrices de longitud cero y no realiza cálculos redundantes. [\#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([chimbab](https://github.com/achimbab)) +- Se corrigió la comprobación de si se permite un host cliente al hacer coincidir el host cliente con `host_regexp` especificado en `users.xml`. [\#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([Vitaly Baranov](https://github.com/vitlibar)) +- Relax comprobación de columna ambigua que conduce a falsos positivos en múltiples `JOIN ON` apartado. [\#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió un posible bloqueo del servidor (`std::terminate`) cuando el servidor no puede enviar o escribir datos en `JSON` o `XML` formato con valores de `String` (tipo de datos que requieren `UTF-8` validación) o al comprimir datos de resultados con el algoritmo Brotli o en algunos otros casos raros. Esto corrige [\#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [\#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir la condición de carrera en `StorageDistributedDirectoryMonitor` encontrado por CI. Esto corrige [\#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [\#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Ahora el fondo se fusiona en `*MergeTree` familia de motores de mesa preservar el orden de volumen de políticas de almacenamiento con mayor precisión. [\#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Ahora motor de mesa `Kafka` funciona correctamente con `Native` formato. Esto corrige [\#6731](https://github.com/ClickHouse/ClickHouse/issues/6731) [\#7337](https://github.com/ClickHouse/ClickHouse/issues/7337) [\#8003](https://github.com/ClickHouse/ClickHouse/issues/8003). [\#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) +- Formatos fijos con encabezados (como `CSVWithNames`) que lanzaban una excepción sobre EOF para el motor de tabla `Kafka`. [\#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) +- Se corrigió un error con la creación de conjunto de subconsulta en la parte derecha de `IN` apartado. Esto corrige [\#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) y [\#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [\#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Solucionar un posible bloqueo al leer desde el almacenamiento `File`. [\#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Lectura fija de los archivos en `Parquet` formato que contiene columnas de tipo `list`. [\#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([Más información](https://github.com/maxulan)) +- Corregir error `Not found column` para consultas distribuidas con `PREWHERE` condición depende de la clave de muestreo si `max_parallel_replicas > 1`. [\#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir error `Not found column` si se utiliza la consulta `PREWHERE` depende del alias de la tabla y el conjunto de resultados estaba vacío debido a la condición de la clave principal. [\#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Tipo de retorno fijo para funciones `rand` y `randConstant` en caso de `Nullable` argumento. Ahora las funciones siempre regresan `UInt32` y nunca `Nullable(UInt32)`. [\#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Inversión de predicado deshabilitada para `WITH FILL` expresion. Esto corrige [\#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [\#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([Invierno Zhang](https://github.com/zhang2014)) +- Corregido incorrecto `count()` resultado para `SummingMergeTree` cuando `FINAL` se utiliza la sección. [\#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [\#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Corregir posible resultado incorrecto para funciones constantes de servidores remotos. Sucedió para consultas con funciones como `version()`, `uptime()`, sucesivamente. que devuelve diferentes valores constantes para diferentes servidores. Esto corrige [\#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [\#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir un error complicado en la optimización de predicados push-down que conduce a resultados incorrectos. Esto soluciona muchos problemas en la optimización de predicados push-down. [\#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([Invierno Zhang](https://github.com/zhang2014)) +- Fix crash en `CREATE TABLE .. AS dictionary` consulta. [\#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) +- Varias mejoras en la gramática ClickHouse `.g4` file. [\#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([taiyang-li](https://github.com/taiyang-li)) +- Corregir un error que conduce a bloqueos en `JOIN`s con mesas con motor `Join`. Esto corrige [\#7556](https://github.com/ClickHouse/ClickHouse/issues/7556) [\#8254](https://github.com/ClickHouse/ClickHouse/issues/8254) [\#7915](https://github.com/ClickHouse/ClickHouse/issues/7915) [\#8100](https://github.com/ClickHouse/ClickHouse/issues/8100). [\#8298](https://github.com/ClickHouse/ClickHouse/pull/8298) ([Artem Zuikov](https://github.com/4ertus2)) +- Repara la recarga de diccionarios redundantes en `CREATE DATABASE`. [\#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) +- Limitar el número máximo de secuencias para leer desde `StorageFile` y `StorageHDFS`. Correcciones https://github.com/ClickHouse/ClickHouse/issues/7650. [\#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) +- Corregir error en `ALTER ... MODIFY ... CODEC` consulta, cuando el usuario especifica tanto la expresión predeterminada como el códec. Fijar [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [\#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([alesapin](https://github.com/alesapin)) +- Corregir error en la fusión de fondo de columnas con `SimpleAggregateFunction(LowCardinality)` tipo. [\#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Tipo fijo comprobar en función `toDateTime64`. [\#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([Vasily Nemkov](https://github.com/Enmk)) +- Ahora el servidor no se bloquea `LEFT` o `FULL JOIN` con y Unirse al motor y no soportado `join_use_nulls` configuración. [\#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) +- Ahora `DROP DICTIONARY IF EXISTS db.dict` la consulta no arroja una excepción si `db` no existe. [\#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) +- Corregir posibles bloqueos en las funciones de la tabla (`file`, `mysql`, `remote`) causado por el uso de la referencia a eliminado `IStorage` objeto. Corregir el análisis incorrecto de las columnas especificadas en la inserción en la función de la tabla. [\#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([Tavplubix](https://github.com/tavplubix)) +- Asegúrese de que la red esté activa antes de comenzar `clickhouse-server`. Esto corrige [\#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [\#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([Zhichang Yu](https://github.com/yuzhichang)) +- Corregir el manejo de los tiempos de espera para conexiones seguras, por lo que las consultas no se cuelgan indefenitamente. Esto corrige [\#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [\#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `clickhouse-copier`contención despedida entre trabajadores concurrentes. [\#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([Más información](https://github.com/dingxiangfei2009)) +- Ahora las mutaciones no omiten las partes adjuntas, incluso si su versión de mutación fuera más grande que la versión de mutación actual. [\#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) [\#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) +- Ignore copias redundantes de `*MergeTree` partes de datos después de pasar a otro disco y reiniciar el servidor. [\#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Fix crash en `FULL JOIN` con `LowCardinality` en `JOIN` clave. [\#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) +- Prohibido usar el nombre de columna más de una vez en la consulta de inserción como `INSERT INTO tbl (x, y, x)`. Esto corrige [\#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [\#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [\#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([alesapin](https://github.com/alesapin)) +- Se agregó respaldo para detectar el número de núcleos de CPU físicos para CPU desconocidas (usando el número de núcleos de CPU lógicos). Esto corrige [\#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [\#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `There's no column` error para columnas materializadas y alias. [\#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) +- Se corrigió el bloqueo de corte cuando `EXISTS` consulta se utilizó sin `TABLE` o `DICTIONARY` calificador. Como `EXISTS t`. Esto corrige [\#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). Este error se introdujo en la versión 19.17. [\#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir error raro con error `"Sizes of columns doesn't match"` que podrían aparecer al usar `SimpleAggregateFunction` columna. [\#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) +- Corregir error donde el usuario con vacío `allow_databases` obtuvo acceso a todas las bases de datos (y lo mismo para `allow_dictionaries`). [\#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) +- Solucionar el bloqueo del cliente cuando el servidor ya está desconectado del cliente. [\#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) +- Fijar `ORDER BY` comportamiento en caso de ordenar por prefijo de clave primaria y sufijo de clave no primaria. [\#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([Anton Popov](https://github.com/CurtizJ)) +- Compruebe si la columna calificada está presente en la tabla. Esto corrige [\#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [\#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) +- Comportamiento fijo con `ALTER MOVE` se ejecuta inmediatamente después de la fusión de acabado se mueve superparte de especificado. Fijar [\#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [\#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Solucionar un posible bloqueo del servidor mientras se usa `UNION` con diferente número de columnas. Fijar [\#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [\#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir el tamaño de la subcadena de resultados para la función `substr` con tamaño negativo. [\#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Ahora el servidor no ejecuta la mutación de parte en `MergeTree` si no hay suficientes hilos libres en el grupo de fondo. [\#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([Tavplubix](https://github.com/tavplubix)) +- Corregir un error tipográfico menor en el formato `UNION ALL` AST. [\#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([Litao91](https://github.com/litao91)) +- Se corrigieron los resultados incorrectos del filtro de floración para los números negativos. Esto corrige [\#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [\#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([Invierno Zhang](https://github.com/zhang2014)) +- Desbordamiento de búfer potencial fijo en descomprimir. El usuario malicioso puede pasar datos comprimidos fabricados que causarán lectura después del búfer. Este problema fue encontrado por Eldar Zaitov del equipo de seguridad de la información de Yandex. [\#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir el resultado incorrecto debido al desbordamiento de enteros en `arrayIntersect`. [\#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Ahora `OPTIMIZE TABLE` La consulta no esperará a que las réplicas sin conexión realicen la operación. [\#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) +- Fijo `ALTER TTL` analizador para `Replicated*MergeTree` tabla. [\#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corrige la comunicación entre el servidor y el cliente, por lo que el servidor lee la información de las tablas temporales después de un error de consulta. [\#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) +- Fijar `bitmapAnd` error de función al intersecar un mapa de bits agregado y un mapa de bits escalar. [\#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) +- Refinar la definición de `ZXid` de acuerdo con la Guía del Programador ZooKeeper que corrige errores en `clickhouse-cluster-copier`. [\#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Más información](https://github.com/dingxiangfei2009)) +- `odbc` la función de la tabla ahora respeta `external_table_functions_use_nulls` configuración. [\#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([Vasily Nemkov](https://github.com/Enmk)) +- Se corrigió un error que conducía a una rara carrera de datos. [\#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alejandro Kazakov](https://github.com/Akazz)) +- Ahora `SYSTEM RELOAD DICTIONARY` recarga un diccionario completamente, ignorando `update_field`. Esto corrige [\#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [\#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([Vitaly Baranov](https://github.com/vitlibar)) +- Agregue la capacidad de verificar si el diccionario existe en la consulta de creación. [\#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([alesapin](https://github.com/alesapin)) +- Fijar `Float*` análisis en `Values` formato. Esto corrige [\#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [\#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([Tavplubix](https://github.com/tavplubix)) +- Solucionar el bloqueo cuando no podemos reservar espacio en algunas operaciones en segundo plano de `*MergeTree` familia de motores de mesa. [\#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Corregir el bloqueo de la operación de fusión cuando la tabla contiene `SimpleAggregateFunction(LowCardinality)` columna. Esto corrige [\#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [\#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) +- Restaure el soporte de todas las configuraciones regionales de la UCI y agregue la capacidad de aplicar intercalaciones para expresiones constantes. También agregue el nombre del idioma a `system.collations` tabla. [\#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) +- Corregir un error cuando los diccionarios externos con una vida útil mínima cero (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`) no actualizar en segundo plano. [\#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([alesapin](https://github.com/alesapin)) +- Solucionar el bloqueo cuando el diccionario externo con la fuente ClickHouse tiene una subconsulta en la consulta. [\#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Corregir el análisis incorrecto de la extensión de archivo en la tabla con el motor `URL`. Esto corrige [\#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [\#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([Andrey Bodrov](https://github.com/apbodrov)) +- Fijar `CHECK TABLE` consulta para `*MergeTree` mesas sin llave. Fijar [\#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [\#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) +- Conversión fija de `Float64` al tipo de MySQL. [\#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([Yuriy Baranov](https://github.com/yurriy)) +- Ahora, si la tabla no se eliminó por completo debido a un bloqueo del servidor, el servidor intentará restaurarla y cargarla. [\#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([Tavplubix](https://github.com/tavplubix)) +- Se corrigió el fallo en la función de la tabla `file` mientras se inserta en el archivo que no existe. Ahora, en este caso, se crearía un archivo y luego se procesaría la inserción. [\#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) +- Repara un punto muerto raro que puede ocurrir cuando `trace_log` está habilitado. [\#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) +- Añadir capacidad de trabajar con diferentes tipos además `Date` en `RangeHashed` diccionario externo creado a partir de consulta DDL. Fijar [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [\#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) +- Corrige el bloqueo cuando `now64()` se llama con el resultado de otra función. [\#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([Vasily Nemkov](https://github.com/Enmk)) +- Se corrigió un error con la detección de IP del cliente para las conexiones a través del protocolo de cable mysql. [\#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) +- Repara el manejo de matrices vacías en `arraySplit` función. Esto corrige [\#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [\#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([Hcz](https://github.com/hczhcz)) +- Se corrigió el problema cuando `pid-file` de otra carrera `clickhouse-server` puede ser eliminado. [\#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([Weiqing Xu](https://github.com/weiqxu)) +- Repara la recarga del diccionario si tiene `invalidate_query`, que detuvo las actualizaciones y alguna excepción en los intentos de actualización anteriores. [\#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) +- Corregido el error en la función `arrayReduce` que puede conducir a “double free” y error en el combinador de funciones agregadas `Resample` que puede conducir a la pérdida de memoria. Función agregada agregada `aggThrow`. Esta función se puede utilizar para fines de prueba. [\#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Mejora {#improvement-1} + +- Registro mejorado cuando se trabaja con `S3` motor de mesa. [\#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +- Mensaje de ayuda impreso cuando no se pasan argumentos al llamar `clickhouse-local`. Esto corrige [\#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [\#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([Andrey Nagorny](https://github.com/Melancholic)) +- Añadir ajuste `mutations_sync` que permite esperar `ALTER UPDATE/DELETE` consultas sincrónicamente. [\#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([alesapin](https://github.com/alesapin)) +- Permitir configurar relativo `user_files_path` en `config.xml` (en la forma similar a `format_schema_path`). [\#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([Hcz](https://github.com/hczhcz)) +- Agregue una excepción para tipos ilegales para funciones de conversión con `-OrZero` postfix. [\#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([Andrey Konyaev](https://github.com/akonyaev90)) +- Simplifique el formato del encabezado de los datos que se envían a un fragmento en una consulta distribuida. [\#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([Vitaly Baranov](https://github.com/vitlibar)) +- `Live View` refactorización del motor de mesa. [\#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) +- Agregue comprobaciones adicionales para diccionarios externos creados a partir de consultas DDL. [\#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([alesapin](https://github.com/alesapin)) +- Corregir error `Column ... already exists` mientras usa `FINAL` y `SAMPLE` together, e.g. `select count() from table final sample 1/2`. Fijar [\#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [\#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Ahora tabla el primer argumento de `joinGet` la función puede ser indentificador de tabla. [\#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([Amos pájaro](https://github.com/amosbird)) +- Permitir usar `MaterializedView` con subconsultas anteriores `Kafka` tabla. [\#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) +- Ahora el fondo se mueve entre discos, ejecuta el grupo de subprocesos seprate. [\#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) +- `SYSTEM RELOAD DICTIONARY` ahora se ejecuta sincrónicamente. [\#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([Vitaly Baranov](https://github.com/vitlibar)) +- Los rastros de pila ahora muestran direcciones físicas (desconectados en el archivo de objeto) en lugar de direcciones de memoria virtual (donde se cargó el archivo de objeto). Eso permite el uso de `addr2line` cuando binary es independiente de la posición y ASLR está activo. Esto corrige [\#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [\#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Soporte nueva sintaxis para filtros de seguridad de nivel de fila: `
`. Fijar [\#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [\#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([Ivan](https://github.com/abyss7)) +- Ahora `cityHash` función puede trabajar con `Decimal` y `UUID` tipo. Fijar [\#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [\#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([Mikhail Korotov](https://github.com/millb)) +- Se eliminó la granularidad de índice fijo (era 1024) de los registros del sistema porque está obsoleto después de la implementación de la granularidad adaptativa. [\#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Habilitado servidor de compatibilidad MySQL cuando ClickHouse se compila sin SSL. [\#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([Yuriy Baranov](https://github.com/yurriy)) +- Ahora las sumas de comprobación del servidor distribuyen lotes, lo que da más errores detallados en caso de datos dañados en el lote. [\#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) +- Apoyo `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` y `ATTACH TABLE` para `MySQL` motor de base de datos. [\#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([Invierno Zhang](https://github.com/zhang2014)) +- Agregue autenticación en la función de tabla S3 y el motor de tabla. [\#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Añadido cheque para piezas adicionales de `MergeTree` en diferentes discos, para no permitir perder partes de datos en discos indefinidos. [\#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Habilite el soporte SSL para el cliente y el servidor de Mac. [\#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([Ivan](https://github.com/abyss7)) +- Ahora ClickHouse puede funcionar como servidor federado de MySQL (consulte https://dev.mysql.com/doc/refman/5.7/en/federated-create-server.html ). [\#7717](https://github.com/ClickHouse/ClickHouse/pull/7717) ([Maxim Fedotov](https://github.com/MaxFedotov)) +- `clickhouse-client` ahora sólo habilitar `bracketed-paste` cuando multiquery está encendido y multilínea está apagado. Esto corrige (\#7757)\[https://github.com/ClickHouse/ClickHouse/issues/7757\]. [\#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([Amos pájaro](https://github.com/amosbird)) +- Apoyo `Array(Decimal)` en `if` función. [\#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) +- Decimales de apoyo en `arrayDifference`, `arrayCumSum` y `arrayCumSumNegative` función. [\#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) +- Añadir `lifetime` columna a `system.dictionaries` tabla. [\#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [\#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) +- Comprobación mejorada de piezas existentes en diferentes discos para `*MergeTree` motores de mesa. Dirección [\#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [\#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Integración con `AWS SDK` para `S3` interacciones que permite utilizar todas las características de S3 fuera de la caja. [\#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([Pavel Kovalenko](https://github.com/Jokser)) +- Se agregó soporte para subconsultas en `Live View` tabla. [\#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) +- Compruebe si el uso `Date` o `DateTime` columna de `TTL` expresiones se eliminó. [\#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Se agregó información sobre el disco a `system.detached_parts` tabla. [\#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Ahora configuración `max_(table|partition)_size_to_drop` se puede cambiar sin reiniciar. [\#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +- Ligeramente mejor usabilidad de los mensajes de error. Pida al usuario que no elimine las siguientes líneas `Stack trace:`. [\#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor lectura de mensajes de `Kafka` motor en varios formatos después de [\#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [\#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([Ivan](https://github.com/abyss7)) +- Mejor compatibilidad con clientes MySQL que no admiten `sha2_password` complemento auth. [\#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([Yuriy Baranov](https://github.com/yurriy)) +- Soporta más tipos de columnas en el servidor de compatibilidad MySQL. [\#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([Yuriy Baranov](https://github.com/yurriy)) +- Implementar `ORDER BY` optimización para `Merge`, `Buffer` y `Materilized View` almacenamientos con subyacente `MergeTree` tabla. [\#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([Anton Popov](https://github.com/CurtizJ)) +- Ahora siempre usamos la implementación POSIX de `getrandom` para tener una mejor compatibilidad con los núcleos antiguos (\<3.17). [\#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([Amos pájaro](https://github.com/amosbird)) +- Es mejor comprobar si hay un destino válido en una regla TTL de movimiento. [\#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Mejores controles de lotes de inserción rotos para `Distributed` motor de mesa. [\#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) +- Agregue una columna con una matriz de nombres de partes para qué mutaciones deben procesar en el futuro `system.mutations` tabla. [\#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([alesapin](https://github.com/alesapin)) +- Optimización de ordenación de combinación paralela para procesadores. [\#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Configuración `mark_cache_min_lifetime` ahora está obsoleto y no hace nada. En versiones anteriores, la caché de marcas puede crecer en la memoria más grande que `mark_cache_size` para acomodar datos dentro de `mark_cache_min_lifetime` segundo. Eso provocó confusión y un mayor uso de memoria de lo esperado, lo que es especialmente malo en los sistemas con restricciones de memoria. Si observa una degradación del rendimiento después de instalar esta versión, debe aumentar la `mark_cache_size`. [\#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Preparación para usar `tid` doquier. Esto es necesario para [\#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [\#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Mejora del rendimiento {#performance-improvement-1} + +- Optimizaciones de rendimiento en la canalización de procesadores. [\#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Actualizaciones sin bloqueo de claves caducadas en diccionarios de caché (con permiso para leer las antiguas). [\#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Compilar ClickHouse sin `-fno-omit-frame-pointer` a nivel mundial para ahorrar un registro más. [\#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([Amos pájaro](https://github.com/amosbird)) +- Aceleración `greatCircleDistance` función y añadir pruebas de rendimiento para ello. [\#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) +- Rendimiento mejorado de la función `roundDown`. [\#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Rendimiento mejorado de `max`, `min`, `argMin`, `argMax` para `DateTime64` tipo de datos. [\#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([Vasily Nemkov](https://github.com/Enmk)) +- Rendimiento mejorado de la clasificación sin límite o con límite grande y clasificación externa. [\#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Rendimiento mejorado de formatear números de coma flotante hasta 6 veces. [\#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Rendimiento mejorado de `modulo` función. [\#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([Amos pájaro](https://github.com/amosbird)) +- Optimizar `ORDER BY` y fusionándose con la clave de una sola columna. [\#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Mejor implementación para `arrayReduce`, `-Array` y `-State` combinadores. [\#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([Amos pájaro](https://github.com/amosbird)) +- Ahora `PREWHERE` debe ser optimizado para ser al menos tan eficiente como `WHERE`. [\#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([Amos pájaro](https://github.com/amosbird)) +- Mejorar el camino `round` y `roundBankers` manejo de números negativos. [\#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([Hcz](https://github.com/hczhcz)) +- Rendimiento de decodificación mejorado de `DoubleDelta` y `Gorilla` códecs en aproximadamente un 30-40%. Esto corrige [\#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [\#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([Vasily Nemkov](https://github.com/Enmk)) +- Rendimiento mejorado de `base64` funciones relacionadas. [\#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se agregó una función `geoDistance`. Es similar a `greatCircleDistance` pero utiliza la aproximación al modelo elipsoide WGS-84. El rendimiento de ambas funciones son casi iguales. [\#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Más rápido `min` y `max` funciones de agregación para `Decimal` tipo de datos. [\#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) +- Vectorizar el procesamiento `arrayReduce`. [\#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos pájaro](https://github.com/amosbird)) +- `if` las cadenas ahora están optimizadas como `multiIf`. [\#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) +- Corregir la regresión de rendimiento de `Kafka` motor de mesa introducido en 19.15. Esto corrige [\#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [\#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) +- Quitar “pie” generación de código que `gcc` de paquetes Debian trae ocasionalmente por defecto. [\#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Formatos de datos de análisis paralelo [\#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +- Habilitar analizador optimizado de `Values` con expresiones por defecto (`input_format_values_deduce_templates_of_expressions=1`). [\#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([Tavplubix](https://github.com/tavplubix)) + +### Mejora de la construcción/prueba/empaquetado {#buildtestingpackaging-improvement-1} + +- Construir correcciones para `ARM` y en modo mínimo. [\#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([propulsor](https://github.com/proller)) +- Añadir archivo de cobertura al ras para `clickhouse-server` cuando std::atexit no se llama. También mejoró ligeramente el registro en pruebas sin estado con cobertura. [\#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([alesapin](https://github.com/alesapin)) +- Actualizar la biblioteca LLVM en contrib. Evite usar LLVM de paquetes de sistema operativo. [\#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Hacer incluido `curl` construir totalmente tranquilo. [\#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [\#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([Pavel Kovalenko](https://github.com/Jokser)) +- Arreglar algunos `MemorySanitizer` advertencia. [\#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Utilizar `add_warning` y `no_warning` macros en `CMakeLists.txt`. [\#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([Ivan](https://github.com/abyss7)) +- Agregue soporte del objeto Compatible Minio S3 (https://min.io/) para mejores pruebas de integración. [\#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [\#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([Pavel Kovalenko](https://github.com/Jokser)) +- Importar `libc` encabezados a contrib. Permite hacer que las compilaciones sean más consistentes en varios sistemas (solo para `x86_64-linux-gnu`). [\#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Quitar `-fPIC` de algunas bibliotecas. [\#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Limpiar `CMakeLists.txt` para el rizo. Ver https://github.com/ClickHouse/ClickHouse/pull/8011\#issuecomment-569478910 [\#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Advertencias silenciosas en `CapNProto` biblioteca. [\#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Agregue pruebas de rendimiento para tablas hash optimizadas de cadenas cortas. [\#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([Amos pájaro](https://github.com/amosbird)) +- Ahora ClickHouse se basará en `AArch64` aunque `MADV_FREE` no está disponible. Esto corrige [\#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [\#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([Amos pájaro](https://github.com/amosbird)) +- Actualizar `zlib-ng` para solucionar problemas de desinfectante de memoria. [\#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [\#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Habilite la biblioteca MySQL interna en sistemas que no son Linux, porque el uso de paquetes de sistemas operativos es muy frágil y generalmente no funciona en absoluto. Esto corrige [\#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [\#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se corrigió la compilación en algunos sistemas después de habilitar `libc++`. Esto reemplaza [\#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [\#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Hacer `Field` métodos más seguros para encontrar más errores. [\#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [\#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Añadido archivos que faltan a la `libc-headers` submódulo. [\#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Arreglar mal `JSON` cita en la salida de prueba de rendimiento. [\#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Ahora se muestra el seguimiento de pila para `std::exception` y `Poco::Exception`. En versiones anteriores sólo estaba disponible para `DB::Exception`. Esto mejora el diagnóstico. [\#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Portabilidad `clock_gettime` y `clock_nanosleep` para versiones glibc frescas. [\#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([Amos pájaro](https://github.com/amosbird)) +- Permitir `part_log` en ejemplo config para desarrolladores. [\#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Corregir la naturaleza asíncrona de la recarga en `01036_no_superfluous_dict_reload_on_create_database*`. [\#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) +- Se corrigieron las pruebas de rendimiento del códec. [\#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([Vasily Nemkov](https://github.com/Enmk)) +- Agregar scripts de instalación para `.tgz` y documentación para ellos. [\#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [\#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([alesapin](https://github.com/alesapin)) +- Eliminado viejo `ZSTD` prueba (fue creado en el año 2016 para reproducir el error que ha tenido la versión pre 1.0 de ZSTD). Esto corrige [\#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [\#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Estructura fija en Mac OS Catalina. [\#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) +- Mayor número de filas en las pruebas de rendimiento del códec para que los resultados se noten. [\#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([Vasily Nemkov](https://github.com/Enmk)) +- En compilaciones de depuración, trate `LOGICAL_ERROR` excepciones como fallas de aserción, por lo que son más fáciles de notar. [\#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Haga que la prueba de rendimiento relacionada con los formatos sea más determinista. [\#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar `lz4` para solucionar un error de MemorySanitizer. [\#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Suprima un falso positivo MemorySanitizer conocido en el control de excepciones. [\#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Actualizar `gcc` y `g++` a la versión 9 en `build/docker/build.sh` [\#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) +- Agregue un caso de prueba de rendimiento para probar eso `PREWHERE` es peor que `WHERE`. [\#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([Amos pájaro](https://github.com/amosbird)) +- Progreso hacia la fijación de una prueba flacky. [\#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Evite el informe MemorySanitizer para los datos de `libunwind`. [\#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Actualizar `libc++` a la última versión. [\#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Construir la biblioteca de la UCI a partir de fuentes. Esto corrige [\#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [\#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cambiado de `libressl` a `openssl`. ClickHouse debe admitir TLS 1.3 y SNI después de este cambio. Esto corrige [\#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [\#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Informe UBSan fijo al usar `chacha20_poly1305` de SSL (sucede al conectarse a https://yandex.ru/). [\#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Modo de reparación del archivo de contraseña predeterminado para `.deb` distribuciones de linux. [\#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([propulsor](https://github.com/proller)) +- Expresión mejorada para obtener `clickhouse-server` PID en `clickhouse-test`. [\#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([Alejandro Kazakov](https://github.com/Akazz)) +- Actualizado contrib / googletest a v1.10.0. [\#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alejandro Burmak](https://github.com/Alex-Burmak)) +- Informe de ThreadSaninitizer fijo en `base64` biblioteca. También actualizó esta biblioteca a la última versión, pero no importa. Esto corrige [\#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [\#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Fijar `00600_replace_running_query` para procesadores. [\#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Eliminar soporte para `tcmalloc` hacer `CMakeLists.txt` más simple. [\#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Lanzamiento gcc construye ahora uso `libc++` en lugar de `libstdc++`. Recientemente `libc++` fue utilizado sólo con clang. Esto mejorará la coherencia de las configuraciones de compilación y la portabilidad. [\#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Habilite la biblioteca de ICU para compilar con MemorySanitizer. [\#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Suprimir advertencias de `CapNProto` biblioteca. [\#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Se eliminaron casos especiales de código para `tcmalloc`, porque ya no es compatible. [\#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- En la tarea de cobertura de CI, elimine el servidor con elegancia para permitirle guardar el informe de cobertura. Esto corrige los informes de cobertura incompletos que hemos estado viendo últimamente. [\#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([alesapin](https://github.com/alesapin)) +- Pruebas de rendimiento para todos los códecs contra `Float64` y `UInt64` valor. [\#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([Vasily Nemkov](https://github.com/Enmk)) +- `termcap` está muy en desuso y conduce a varios problemas (por ejemplo, falta “up” tapa y eco `^J` en lugar de la línea multi). Favor `terminfo` o empaquetado `ncurses`. [\#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([Amos pájaro](https://github.com/amosbird)) +- Fijar `test_storage_s3` prueba de integración. [\#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Apoyo `StorageFile(, null)` para insertar el bloque en el archivo de formato dado sin realmente escribir en el disco. Esto es necesario para las pruebas de rendimiento. [\#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos pájaro](https://github.com/amosbird)) +- Argumento añadido `--print-time` a las pruebas funcionales que imprime el tiempo de ejecución por prueba. [\#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Se agregaron afirmaciones a `KeyCondition` mientras evalúa RPN. Esto arreglará la advertencia de gcc-9. [\#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Opciones de volcado cmake en compilaciones de CI. [\#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- No genere información de depuración para algunas bibliotecas de grasa. [\#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Hacer `log_to_console.xml` siempre inicie sesión en stderr, independientemente de si es interactivo o no. [\#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alejandro Kuzmenkov](https://github.com/akuzm)) +- Se eliminaron algunas características no utilizadas de `clickhouse-performance-test` herramienta. [\#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Ahora también buscaremos `lld-X` con la correspondiente `clang-X` versión. [\#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([alesapin](https://github.com/alesapin)) +- Parquet construir mejora. [\#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([Más información](https://github.com/maxulan)) +- Más advertencias de GCC [\#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([Método de codificación de datos:](https://github.com/kreuzerkrieg)) +- Paquete para Arch Linux ahora permite ejecutar el servidor ClickHouse, y no sólo el cliente. [\#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) +- Prueba de reparación con procesadores. Pequeñas correcciones de rendimiento. [\#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +- Actualizar contrib/protobuf. [\#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V. Kornilov](https://github.com/matwey)) +- En preparación para cambiar a c ++ 20 como una celebración de año nuevo. “May the C++ force be with ClickHouse.” [\#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([Amos pájaro](https://github.com/amosbird)) + +### Característica Experimental {#experimental-feature-1} + +- Añadido ajuste experimental `min_bytes_to_use_mmap_io`. Permite leer archivos grandes sin copiar datos del kernel al espacio de usuario. La configuración está deshabilitada de forma predeterminada. El umbral recomendado es de aproximadamente 64 MB, porque mmap / munmap es lento. [\#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([alexey-milovidov](https://github.com/alexey-milovidov)) +- Cuotas reanudadas como parte del sistema de control de acceso. Añadido nueva tabla `system.quotas`, nuevas funciones `currentQuota`, `currentQuotaKey`, nueva sintaxis SQL `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [\#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([Vitaly Baranov](https://github.com/vitlibar)) +- Permitir omitir configuraciones desconocidas con advertencias en lugar de lanzar excepciones. [\#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([Vitaly Baranov](https://github.com/vitlibar)) +- Se reanudaron las políticas de fila como parte del sistema de control de acceso. Añadido nueva tabla `system.row_policies`, nueva función `currentRowPolicies()`, nueva sintaxis SQL `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [\#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([Vitaly Baranov](https://github.com/vitlibar)) + +### Corrección de seguridad {#security-fix} + +- Se corrigió la posibilidad de leer la estructura de directorios en tablas con `File` motor de mesa. Esto corrige [\#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [\#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([alexey-milovidov](https://github.com/alexey-milovidov)) diff --git a/docs/es/commercial/cloud.md b/docs/es/commercial/cloud.md index ee6a45fc851..f77f536341f 100644 --- a/docs/es/commercial/cloud.md +++ b/docs/es/commercial/cloud.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Proveedores de servicios en la nube de ClickHouse {#clickhouse-cloud-service-providers} !!! info "INFO" diff --git a/docs/es/data_types/array.md b/docs/es/data_types/array.md index 1650ab461b1..bd6fd4f65a7 100644 --- a/docs/es/data_types/array.md +++ b/docs/es/data_types/array.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Matriz (T) {#data-type-array} Matriz de `T`-tipo de artículos. diff --git a/docs/es/data_types/boolean.md b/docs/es/data_types/boolean.md index 3a49cbd7658..7f7d839dcdb 100644 --- a/docs/es/data_types/boolean.md +++ b/docs/es/data_types/boolean.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Valores booleanos {#boolean-values} No hay un tipo separado para los valores booleanos. Utilice el tipo UInt8, restringido a los valores 0 o 1. diff --git a/docs/es/data_types/date.md b/docs/es/data_types/date.md index 78d265ab604..3a4f5cfb5d9 100644 --- a/docs/es/data_types/date.md +++ b/docs/es/data_types/date.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Fecha {#date} Fecha. Almacenado en dos bytes como el número de días desde 1970-01-01 (sin signo). Permite almacenar valores desde justo después del comienzo de la Época Unix hasta el umbral superior definido por una constante en la etapa de compilación (actualmente, esto es hasta el año 2106, pero el último año totalmente soportado es 2105). diff --git a/docs/es/data_types/datetime.md b/docs/es/data_types/datetime.md index ab91a510cba..0f8e35634ca 100644 --- a/docs/es/data_types/datetime.md +++ b/docs/es/data_types/datetime.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # FechaHora {#data_type-datetime} Permite almacenar un instante en el tiempo, que se puede expresar como una fecha del calendario y una hora de un día. diff --git a/docs/es/data_types/datetime64.md b/docs/es/data_types/datetime64.md index 444a6357b07..e9a5245b146 100644 --- a/docs/es/data_types/datetime64.md +++ b/docs/es/data_types/datetime64.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # DateTime64 {#data_type-datetime64} Permite almacenar un instante en el tiempo, que se puede expresar como una fecha de calendario y una hora de un día, con una precisión de subsegundo definida diff --git a/docs/es/data_types/decimal.md b/docs/es/data_types/decimal.md index 2a355ed79a9..9e84975443e 100644 --- a/docs/es/data_types/decimal.md +++ b/docs/es/data_types/decimal.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) {#decimalp-s-decimal32s-decimal64s-decimal128s} Números de punto fijo firmados que mantienen la precisión durante las operaciones de suma, resta y multiplicación. Para la división se descartan los dígitos menos significativos (no redondeados). diff --git a/docs/es/data_types/domains/ipv4.md b/docs/es/data_types/domains/ipv4.md index b2f56b15659..9e3dcbd5167 100644 --- a/docs/es/data_types/domains/ipv4.md +++ b/docs/es/data_types/domains/ipv4.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ## IPv4 {#ipv4} `IPv4` es un dominio basado en `UInt32` tipo y sirve como reemplazo tipo para almacenar valores IPv4. Proporciona un almacenamiento compacto con un formato de entrada-salida amigable para los humanos e información de tipo de columna sobre la inspección. diff --git a/docs/es/data_types/domains/ipv6.md b/docs/es/data_types/domains/ipv6.md index f385774ee04..fa9bd746b30 100644 --- a/docs/es/data_types/domains/ipv6.md +++ b/docs/es/data_types/domains/ipv6.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ## IPv6 {#ipv6} `IPv6` es un dominio basado en `FixedString(16)` tipo y sirve como reemplazo tipo para almacenar valores IPv6. Proporciona un almacenamiento compacto con un formato de entrada-salida amigable para los humanos e información de tipo de columna sobre la inspección. diff --git a/docs/es/data_types/domains/overview.md b/docs/es/data_types/domains/overview.md index 378b7b02997..2d662c4f62f 100644 --- a/docs/es/data_types/domains/overview.md +++ b/docs/es/data_types/domains/overview.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Dominio {#domains} Los dominios son tipos de propósito especial, que agregan algunas características adicionales encima del tipo base existente, dejando intacto el formato en cable y en disco de la tabla subyacente. Por el momento, ClickHouse no admite dominios definidos por el usuario. diff --git a/docs/es/data_types/enum.md b/docs/es/data_types/enum.md index 2c66fa20d31..dd6ff216a52 100644 --- a/docs/es/data_types/enum.md +++ b/docs/es/data_types/enum.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Enum {#enum} Tipo enumerado que consta de valores con nombre. diff --git a/docs/es/data_types/fixedstring.md b/docs/es/data_types/fixedstring.md index 7e02cf6eb48..51aa56239a5 100644 --- a/docs/es/data_types/fixedstring.md +++ b/docs/es/data_types/fixedstring.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Cadena fija {#fixedstring} Una cadena de longitud fija de `N` bytes (ni caracteres ni puntos de código). diff --git a/docs/es/data_types/float.md b/docs/es/data_types/float.md index 58a42f8b56a..7ce233b612b 100644 --- a/docs/es/data_types/float.md +++ b/docs/es/data_types/float.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Descripción del producto {#float32-float64} [Números de punto flotante](https://en.wikipedia.org/wiki/IEEE_754). diff --git a/docs/es/data_types/index.md b/docs/es/data_types/index.md index 8df6bea6314..f787476f864 100644 --- a/docs/es/data_types/index.md +++ b/docs/es/data_types/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Tipos de datos {#data_types} ClickHouse puede almacenar varios tipos de datos en celdas de tabla. diff --git a/docs/es/data_types/int_uint.md b/docs/es/data_types/int_uint.md index 949b490fc99..ee64697f2ef 100644 --- a/docs/es/data_types/int_uint.md +++ b/docs/es/data_types/int_uint.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} Enteros de longitud fija, con o sin signo. diff --git a/docs/es/data_types/nested_data_structures/aggregatefunction.md b/docs/es/data_types/nested_data_structures/aggregatefunction.md index 9c496b8ec1e..a5e2214853e 100644 --- a/docs/es/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/es/data_types/nested_data_structures/aggregatefunction.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # AggregateFunction(name, types\_of\_arguments…) {#data-type-aggregatefunction} El estado intermedio de una función agregada. Para obtenerlo, use funciones agregadas con el `-State` sufijo. Para obtener datos agregados en el futuro, debe utilizar las mismas funciones agregadas con el `-Merge`sufijo. diff --git a/docs/es/data_types/nested_data_structures/index.md b/docs/es/data_types/nested_data_structures/index.md index dac0e7417c4..9e5e2ad5444 100644 --- a/docs/es/data_types/nested_data_structures/index.md +++ b/docs/es/data_types/nested_data_structures/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Estructuras de datos anidados {#nested-data-structures} [Artículo Original](https://clickhouse.tech/docs/es/data_types/nested_data_structures/) diff --git a/docs/es/data_types/nested_data_structures/nested.md b/docs/es/data_types/nested_data_structures/nested.md index 1ea1e777ade..b316a29584b 100644 --- a/docs/es/data_types/nested_data_structures/nested.md +++ b/docs/es/data_types/nested_data_structures/nested.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nested(Name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2} A nested data structure is like a nested table. The parameters of a nested data structure – the column names and types – are specified the same way as in a CREATE query. Each table row can correspond to any number of rows in a nested data structure. diff --git a/docs/es/data_types/nullable.md b/docs/es/data_types/nullable.md index 327bb9d335a..9208ff90713 100644 --- a/docs/es/data_types/nullable.md +++ b/docs/es/data_types/nullable.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nombre de tipo) {#data_type-nullable} Permite almacenar marcador especial ([NULO](../query_language/syntax.md)) que denota “missing value” con los valores normales permitidos por `TypeName`. Por ejemplo, un `Nullable(Int8)` Tipo columna puede almacenar `Int8` valores de tipo, y las filas que no tienen un valor almacenarán `NULL`. diff --git a/docs/es/data_types/special_data_types/expression.md b/docs/es/data_types/special_data_types/expression.md index c1da3d79455..bfdbf3ed0b1 100644 --- a/docs/es/data_types/special_data_types/expression.md +++ b/docs/es/data_types/special_data_types/expression.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Expresion {#expression} Se utiliza para representar expresiones lambda en funciones de orden superior. diff --git a/docs/es/data_types/special_data_types/index.md b/docs/es/data_types/special_data_types/index.md index 8c02cb2b23f..5915b7b3d86 100644 --- a/docs/es/data_types/special_data_types/index.md +++ b/docs/es/data_types/special_data_types/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Tipos de datos especiales {#special-data-types} Special data type values can’t be saved to a table or output in results, but can be used as the intermediate result when running a query. diff --git a/docs/es/data_types/special_data_types/interval.md b/docs/es/data_types/special_data_types/interval.md index a95bbacbd8b..099b3d064a5 100644 --- a/docs/es/data_types/special_data_types/interval.md +++ b/docs/es/data_types/special_data_types/interval.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Intervalo {#data-type-interval} La familia de tipos de datos que representan intervalos de fecha y hora. Los tipos resultantes del [INTERVALO](../../query_language/operators.md#operator-interval) operador. diff --git a/docs/es/data_types/special_data_types/nothing.md b/docs/es/data_types/special_data_types/nothing.md index 6f6c8e5f310..2f8b44e2c79 100644 --- a/docs/es/data_types/special_data_types/nothing.md +++ b/docs/es/data_types/special_data_types/nothing.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nada {#nothing} El único propósito de este tipo de datos es representar casos en los que no se espera un valor. Entonces no puedes crear un `Nothing` valor de tipo. diff --git a/docs/es/data_types/special_data_types/set.md b/docs/es/data_types/special_data_types/set.md index 2e3112813f6..5557b20dde2 100644 --- a/docs/es/data_types/special_data_types/set.md +++ b/docs/es/data_types/special_data_types/set.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Establecer {#set} Utilizado para la mitad derecha de un [ES](../../query_language/select.md#select-in-operators) expresión. diff --git a/docs/es/data_types/string.md b/docs/es/data_types/string.md index e86e8fbdab7..238de32fa22 100644 --- a/docs/es/data_types/string.md +++ b/docs/es/data_types/string.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Cadena {#string} Cuerdas de una longitud arbitraria. La longitud no está limitada. El valor puede contener un conjunto arbitrario de bytes, incluidos los bytes nulos. diff --git a/docs/es/data_types/tuple.md b/docs/es/data_types/tuple.md index 41121c4481c..68af5e5a77d 100644 --- a/docs/es/data_types/tuple.md +++ b/docs/es/data_types/tuple.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Tuple (T1, T2, …) {#tuplet1-t2} Una tupla de elementos, cada uno con un individuo [Tipo](index.md#data_types). diff --git a/docs/es/data_types/uuid.md b/docs/es/data_types/uuid.md index a847bb82eb6..1c1910edbf1 100644 --- a/docs/es/data_types/uuid.md +++ b/docs/es/data_types/uuid.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # UUID {#uuid-data-type} Un identificador único universal (UUID) es un número de 16 bytes utilizado para identificar registros. Para obtener información detallada sobre el UUID, consulte [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). diff --git a/docs/es/database_engines/index.md b/docs/es/database_engines/index.md index 635f19d7d66..55ae226a20d 100644 --- a/docs/es/database_engines/index.md +++ b/docs/es/database_engines/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Motores de base de datos {#database-engines} Los motores de bases de datos le permiten trabajar con tablas. diff --git a/docs/es/database_engines/lazy.md b/docs/es/database_engines/lazy.md index 4a39ad2cc8c..95f69695112 100644 --- a/docs/es/database_engines/lazy.md +++ b/docs/es/database_engines/lazy.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/database_engines/mysql.md b/docs/es/database_engines/mysql.md index 9d61b8828ee..927eae65b77 100644 --- a/docs/es/database_engines/mysql.md +++ b/docs/es/database_engines/mysql.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/development/architecture.md b/docs/es/development/architecture.md index a68ba02ab00..a6f41493f26 100644 --- a/docs/es/development/architecture.md +++ b/docs/es/development/architecture.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. Esto se llama “vectorized query execution,” y ayuda a reducir el costo del procesamiento de datos real. diff --git a/docs/es/development/browse_code.md b/docs/es/development/browse_code.md index f292f0edf06..d3d4e54f264 100644 --- a/docs/es/development/browse_code.md +++ b/docs/es/development/browse_code.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/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. diff --git a/docs/es/development/build.md b/docs/es/development/build.md index 8ae379338a9..9e4ab7628f2 100644 --- a/docs/es/development/build.md +++ b/docs/es/development/build.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Cómo construir ClickHouse para el desarrollo {#how-to-build-clickhouse-for-development} El siguiente tutorial se basa en el sistema Ubuntu Linux. diff --git a/docs/es/development/build_cross_arm.md b/docs/es/development/build_cross_arm.md index c09fd0bc151..c7e8da0d864 100644 --- a/docs/es/development/build_cross_arm.md +++ b/docs/es/development/build_cross_arm.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/development/build_cross_osx.md b/docs/es/development/build_cross_osx.md index 062a049bd53..584b28caac7 100644 --- a/docs/es/development/build_cross_osx.md +++ b/docs/es/development/build_cross_osx.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/development/build_osx.md b/docs/es/development/build_osx.md index 653046fc221..d311411930e 100644 --- a/docs/es/development/build_osx.md +++ b/docs/es/development/build_osx.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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) diff --git a/docs/es/development/contrib.md b/docs/es/development/contrib.md index 456a38fc1ca..d15168b93dd 100644 --- a/docs/es/development/contrib.md +++ b/docs/es/development/contrib.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Bibliotecas de terceros utilizadas {#third-party-libraries-used} | Biblioteca | Licencia | diff --git a/docs/es/development/developer_instruction.md b/docs/es/development/developer_instruction.md index 17fc8bb7d90..62d22eecb9e 100644 --- a/docs/es/development/developer_instruction.md +++ b/docs/es/development/developer_instruction.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + La construcción de ClickHouse es compatible con Linux, FreeBSD y Mac OS X. # Si utiliza Windows {#if-you-use-windows} diff --git a/docs/es/development/index.md b/docs/es/development/index.md index c7d13caf3cd..a905fc295b9 100644 --- a/docs/es/development/index.md +++ b/docs/es/development/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Desarrollo de ClickHouse {#clickhouse-development} [Artículo Original](https://clickhouse.tech/docs/es/development/) diff --git a/docs/es/development/style.md b/docs/es/development/style.md index 207ebe65f13..2ba57bc0f63 100644 --- a/docs/es/development/style.md +++ b/docs/es/development/style.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Cómo escribir código C ++ {#how-to-write-c-code} ## Recomendaciones generales {#general-recommendations} diff --git a/docs/es/development/tests.md b/docs/es/development/tests.md index 9682682a9df..1cfd6d253c0 100644 --- a/docs/es/development/tests.md +++ b/docs/es/development/tests.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Pruebas de ClickHouse {#clickhouse-testing} ## Pruebas funcionales {#functional-tests} diff --git a/docs/es/faq/general.md b/docs/es/faq/general.md index 580ae7d1241..351b0f18ec0 100644 --- a/docs/es/faq/general.md +++ b/docs/es/faq/general.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Preguntas generales {#general-questions} ## ¿Por qué no usar algo como MapReduce? {#why-not-use-something-like-mapreduce} diff --git a/docs/es/getting_started/example_datasets/amplab_benchmark.md b/docs/es/getting_started/example_datasets/amplab_benchmark.md index 45dc417ae46..b59d3ff923d 100644 --- a/docs/es/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/es/getting_started/example_datasets/amplab_benchmark.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Referencia de Big Data de AMPLab {#amplab-big-data-benchmark} Ver https://amplab.cs.berkeley.edu/benchmark/ diff --git a/docs/es/getting_started/example_datasets/criteo.md b/docs/es/getting_started/example_datasets/criteo.md index 89922f00679..37f4a5589b6 100644 --- a/docs/es/getting_started/example_datasets/criteo.md +++ b/docs/es/getting_started/example_datasets/criteo.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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/ diff --git a/docs/es/getting_started/example_datasets/index.md b/docs/es/getting_started/example_datasets/index.md new file mode 120000 index 00000000000..c891314f915 --- /dev/null +++ b/docs/es/getting_started/example_datasets/index.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/index.md \ No newline at end of file diff --git a/docs/es/getting_started/example_datasets/metrica.md b/docs/es/getting_started/example_datasets/metrica.md index 187d0724862..d5d8067333c 100644 --- a/docs/es/getting_started/example_datasets/metrica.md +++ b/docs/es/getting_started/example_datasets/metrica.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/getting_started/example_datasets/nyc_taxi.md b/docs/es/getting_started/example_datasets/nyc_taxi.md index 9d2499e2f42..a0fa9ba13cf 100644 --- a/docs/es/getting_started/example_datasets/nyc_taxi.md +++ b/docs/es/getting_started/example_datasets/nyc_taxi.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Datos de taxis de Nueva York {#new-york-taxi-data} Este conjunto de datos se puede obtener de dos maneras: diff --git a/docs/es/getting_started/example_datasets/ontime.md b/docs/es/getting_started/example_datasets/ontime.md index a3d22cf747a..671132dbed5 100644 --- a/docs/es/getting_started/example_datasets/ontime.md +++ b/docs/es/getting_started/example_datasets/ontime.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Un tiempo {#ontime} Este conjunto de datos se puede obtener de dos maneras: diff --git a/docs/es/getting_started/example_datasets/star_schema.md b/docs/es/getting_started/example_datasets/star_schema.md index 45a2d1a76e9..88339d02c6f 100644 --- a/docs/es/getting_started/example_datasets/star_schema.md +++ b/docs/es/getting_started/example_datasets/star_schema.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Estrella Schema Benchmark {#star-schema-benchmark} Compilación de dbgen: diff --git a/docs/es/getting_started/example_datasets/wikistat.md b/docs/es/getting_started/example_datasets/wikistat.md index 92ecf267a7b..c905dc9f742 100644 --- a/docs/es/getting_started/example_datasets/wikistat.md +++ b/docs/es/getting_started/example_datasets/wikistat.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nombre de la red inalámbrica (SSID): {#wikistat} Ver: http://dumps.wikimedia.org/other/pagecounts-raw/ diff --git a/docs/es/getting_started/index.md b/docs/es/getting_started/index.md index 54db5af4ebf..11bc13f2bc6 100644 --- a/docs/es/getting_started/index.md +++ b/docs/es/getting_started/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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: diff --git a/docs/es/getting_started/install.md b/docs/es/getting_started/install.md index 1b93e4ac93e..014afc99ce5 100644 --- a/docs/es/getting_started/install.md +++ b/docs/es/getting_started/install.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Instalación {#installation} ## Requisitos del sistema {#system-requirements} diff --git a/docs/es/getting_started/playground.md b/docs/es/getting_started/playground.md new file mode 120000 index 00000000000..de5b41f453e --- /dev/null +++ b/docs/es/getting_started/playground.md @@ -0,0 +1 @@ +../../en/getting_started/playground.md \ No newline at end of file diff --git a/docs/es/getting_started/tutorial.md b/docs/es/getting_started/tutorial.md index 28c31cc1214..3516b70ee8c 100644 --- a/docs/es/getting_started/tutorial.md +++ b/docs/es/getting_started/tutorial.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Tutorial de ClickHouse {#clickhouse-tutorial} ## Qué Esperar de Este Tutorial? {#what-to-expect-from-this-tutorial} diff --git a/docs/es/guides/apply_catboost_model.md b/docs/es/guides/apply_catboost_model.md index aa717e53da6..7c0e2677c82 100644 --- a/docs/es/guides/apply_catboost_model.md +++ b/docs/es/guides/apply_catboost_model.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/guides/index.md b/docs/es/guides/index.md index 953251d1b17..178e9562ab9 100644 --- a/docs/es/guides/index.md +++ b/docs/es/guides/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Guías de ClickHouse {#clickhouse-guides} Instrucciones detalladas paso a paso que lo ayudarán a resolver varias tareas usando ClickHouse. diff --git a/docs/es/index.md b/docs/es/index.md index b777996b67d..9f9165b446b 100644 --- a/docs/es/index.md +++ b/docs/es/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # ¿Qué es ClickHouse? {#what-is-clickhouse} ClickHouse es un sistema de gestión de bases de datos orientado a columnas (DBMS) para el procesamiento analítico en línea de consultas (OLAP). diff --git a/docs/es/interfaces/cli.md b/docs/es/interfaces/cli.md index 7f035970fac..a0dbd31824b 100644 --- a/docs/es/interfaces/cli.md +++ b/docs/es/interfaces/cli.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/interfaces/cpp.md b/docs/es/interfaces/cpp.md index 9de79433ba7..5fad7c315e5 100644 --- a/docs/es/interfaces/cpp.md +++ b/docs/es/interfaces/cpp.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Biblioteca de clientes de C++ {#c-client-library} Ver README en [Bienvenidos](https://github.com/ClickHouse/clickhouse-cpp) repositorio. diff --git a/docs/es/interfaces/formats.md b/docs/es/interfaces/formats.md deleted file mode 120000 index 41a65ebe579..00000000000 --- a/docs/es/interfaces/formats.md +++ /dev/null @@ -1 +0,0 @@ -../../en/interfaces/formats.md \ No newline at end of file diff --git a/docs/es/interfaces/formats.md b/docs/es/interfaces/formats.md new file mode 100644 index 00000000000..d78c81a2849 --- /dev/null +++ b/docs/es/interfaces/formats.md @@ -0,0 +1,1209 @@ +--- +machine_translated: true +--- + +# 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 + +0000-00-00 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](../query_language/syntax.md) se formatea como `\N`. + +Cada elemento de [Anidar](../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](../query_language/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](../query_language/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](../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](../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](../query_language/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─┐ +│ 0000-00-00 │ 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](../query_language/syntax.md#null-literal) soporte, se añade un byte adicional que contiene 1 o 0 antes de cada [NULL](../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](../query_language/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](../query_language/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](../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](../query_language/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](../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?)](../data_types/int_uint.md), [UInt(8\|16\|32)](../data_types/int_uint.md) | `int` | +| `boolean`, `int`, `long`, `float`, `double` | [Int64](../data_types/int_uint.md), [UInt64](../data_types/int_uint.md) | `long` | +| `boolean`, `int`, `long`, `float`, `double` | [Float32](../data_types/float.md) | `float` | +| `boolean`, `int`, `long`, `float`, `double` | [Float64](../data_types/float.md) | `double` | +| `bytes`, `string`, `fixed`, `enum` | [Cadena](../data_types/string.md) | `bytes` | +| `bytes`, `string`, `fixed` | [Cadena fija (N)](../data_types/fixedstring.md) | `fixed(N)` | +| `enum` | [Enum (8\|16)](../data_types/enum.md) | `enum` | +| `array(T)` | [Matriz (T)](../data_types/array.md) | `array(T)` | +| `union(null, T)`, `union(T, null)` | [Nivel de Cifrado WEP)](../data_types/date.md) | `union(null, T)` | +| `null` | [Nullable (nada)](../data_types/special_data_types/nothing.md) | `null` | +| `int (date)` \* | [Fecha](../data_types/date.md) | `int (date)` \* | +| `long (timestamp-millis)` \* | [¿Qué puedes encontrar en Neodigit)](../data_types/datetime.md) | `long (timestamp-millis)` \* | +| `long (timestamp-micros)` \* | [Cómo hacer esto?)](../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](../query_language/functions/type_conversion_functions/#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/utils/clickhouse-local.md): + +``` 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](../operations/table_engines/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](../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](../data_types/int_uint.md) | `UINT8` | +| `INT8` | [Int8](../data_types/int_uint.md) | `INT8` | +| `UINT16` | [UInt16](../data_types/int_uint.md) | `UINT16` | +| `INT16` | [Int16](../data_types/int_uint.md) | `INT16` | +| `UINT32` | [UInt32](../data_types/int_uint.md) | `UINT32` | +| `INT32` | [Int32](../data_types/int_uint.md) | `INT32` | +| `UINT64` | [UInt64](../data_types/int_uint.md) | `UINT64` | +| `INT64` | [Int64](../data_types/int_uint.md) | `INT64` | +| `FLOAT`, `HALF_FLOAT` | [Float32](../data_types/float.md) | `FLOAT` | +| `DOUBLE` | [Float64](../data_types/float.md) | `DOUBLE` | +| `DATE32` | [Fecha](../data_types/date.md) | `UINT16` | +| `DATE64`, `TIMESTAMP` | [FechaHora](../data_types/datetime.md) | `UINT32` | +| `STRING`, `BINARY` | [Cadena](../data_types/string.md) | `STRING` | +| — | [Cadena fija](../data_types/fixedstring.md) | `STRING` | +| `DECIMAL` | [Decimal](../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](../operations/table_engines/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](../data_types/index.md) en `INSERT` consulta. + +| Tipo de datos ORC (`INSERT`) | Tipo de datos ClickHouse | +|------------------------------|----------------------------------------| +| `UINT8`, `BOOL` | [UInt8](../data_types/int_uint.md) | +| `INT8` | [Int8](../data_types/int_uint.md) | +| `UINT16` | [UInt16](../data_types/int_uint.md) | +| `INT16` | [Int16](../data_types/int_uint.md) | +| `UINT32` | [UInt32](../data_types/int_uint.md) | +| `INT32` | [Int32](../data_types/int_uint.md) | +| `UINT64` | [UInt64](../data_types/int_uint.md) | +| `INT64` | [Int64](../data_types/int_uint.md) | +| `FLOAT`, `HALF_FLOAT` | [Float32](../data_types/float.md) | +| `DOUBLE` | [Float64](../data_types/float.md) | +| `DATE32` | [Fecha](../data_types/date.md) | +| `DATE64`, `TIMESTAMP` | [FechaHora](../data_types/datetime.md) | +| `STRING`, `BINARY` | [Cadena](../data_types/string.md) | +| `DECIMAL` | [Decimal](../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](../query_language/functions/type_conversion_functions/#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](../operations/table_engines/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_settings/settings.md#server_settings-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 index 5d45c3755e5..1097d787a51 100644 --- a/docs/es/interfaces/http.md +++ b/docs/es/interfaces/http.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/interfaces/index.md b/docs/es/interfaces/index.md index e0b5257a280..2bc67a3425c 100644 --- a/docs/es/interfaces/index.md +++ b/docs/es/interfaces/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Interfaz {#interfaces} ClickHouse proporciona dos interfaces de red (ambas se pueden ajustar opcionalmente en TLS para mayor seguridad): diff --git a/docs/es/interfaces/jdbc.md b/docs/es/interfaces/jdbc.md index 3f155da865e..2069c78dff2 100644 --- a/docs/es/interfaces/jdbc.md +++ b/docs/es/interfaces/jdbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Controlador JDBC {#jdbc-driver} - **[Conductor oficial](https://github.com/ClickHouse/clickhouse-jdbc)** diff --git a/docs/es/interfaces/mysql.md b/docs/es/interfaces/mysql.md index cedbac38e4d..bd3e56cf61c 100644 --- a/docs/es/interfaces/mysql.md +++ b/docs/es/interfaces/mysql.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Interfaz de MySQL {#mysql-interface} ClickHouse soporta el protocolo de cable MySQL. Puede ser habilitado por [mysql\_port](../operations/server_settings/settings.md#server_settings-mysql_port) configuración en el archivo de configuración: diff --git a/docs/es/interfaces/odbc.md b/docs/es/interfaces/odbc.md index 8e221d4919c..e5ca03e61d2 100644 --- a/docs/es/interfaces/odbc.md +++ b/docs/es/interfaces/odbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Conductor ODBC {#odbc-driver} - [Conductor oficial](https://github.com/ClickHouse/clickhouse-odbc). diff --git a/docs/es/interfaces/tcp.md b/docs/es/interfaces/tcp.md index 87f91719463..660f506ee04 100644 --- a/docs/es/interfaces/tcp.md +++ b/docs/es/interfaces/tcp.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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/dbms/src/Client)) y/o mediante la interceptación y el análisis del tráfico TCP. diff --git a/docs/es/interfaces/third-party/client_libraries.md b/docs/es/interfaces/third-party/client_libraries.md index ff66962f87b..96dd21d1f9d 100644 --- a/docs/es/interfaces/third-party/client_libraries.md +++ b/docs/es/interfaces/third-party/client_libraries.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Bibliotecas de clientes de desarrolladores de terceros {#client-libraries-from-third-party-developers} !!! warning "Descargo" diff --git a/docs/es/interfaces/third-party/gui.md b/docs/es/interfaces/third-party/gui.md index 1148825c0f9..909b078505d 100644 --- a/docs/es/interfaces/third-party/gui.md +++ b/docs/es/interfaces/third-party/gui.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Interfaces visuales de desarrolladores de terceros {#visual-interfaces-from-third-party-developers} ## De código abierto {#open-source} diff --git a/docs/es/interfaces/third-party/integrations.md b/docs/es/interfaces/third-party/integrations.md index 2a88fafa5be..2d6e525e0bf 100644 --- a/docs/es/interfaces/third-party/integrations.md +++ b/docs/es/interfaces/third-party/integrations.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Bibliotecas de integración de desarrolladores externos {#integration-libraries-from-third-party-developers} !!! warning "Descargo" diff --git a/docs/es/interfaces/third-party/proxy.md b/docs/es/interfaces/third-party/proxy.md index 49f165a4a09..29e0773e3d8 100644 --- a/docs/es/interfaces/third-party/proxy.md +++ b/docs/es/interfaces/third-party/proxy.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Servidores proxy de desarrolladores de terceros {#proxy-servers-from-third-party-developers} ## chproxy {#chproxy} diff --git a/docs/es/introduction/adopters.md b/docs/es/introduction/adopters.md index eab07cbe893..5d23e3f1017 100644 --- a/docs/es/introduction/adopters.md +++ b/docs/es/introduction/adopters.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Adoptadores de ClickHouse {#clickhouse-adopters} !!! warning "Descargo" diff --git a/docs/es/introduction/distinctive_features.md b/docs/es/introduction/distinctive_features.md index 0e53b8935cd..ed071705e8e 100644 --- a/docs/es/introduction/distinctive_features.md +++ b/docs/es/introduction/distinctive_features.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Características distintivas de ClickHouse {#distinctive-features-of-clickhouse} ## DBMS orientado a columnas verdaderas {#true-column-oriented-dbms} diff --git a/docs/es/introduction/features_considered_disadvantages.md b/docs/es/introduction/features_considered_disadvantages.md index 0a9f56083c4..1e615f9caaf 100644 --- a/docs/es/introduction/features_considered_disadvantages.md +++ b/docs/es/introduction/features_considered_disadvantages.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Características de ClickHouse que pueden considerarse desventajas {#clickhouse-features-that-can-be-considered-disadvantages} 1. No hay transacciones completas. diff --git a/docs/es/introduction/history.md b/docs/es/introduction/history.md index 469b084454b..bd50e867d29 100644 --- a/docs/es/introduction/history.md +++ b/docs/es/introduction/history.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Historial de ClickHouse {#clickhouse-history} ClickHouse fue desarrollado originalmente 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. diff --git a/docs/es/introduction/performance.md b/docs/es/introduction/performance.md index a819e285369..4c2dfa971e0 100644 --- a/docs/es/introduction/performance.md +++ b/docs/es/introduction/performance.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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.html). diff --git a/docs/es/operations/access_rights.md b/docs/es/operations/access_rights.md index 1e60285c259..9006ade40dd 100644 --- a/docs/es/operations/access_rights.md +++ b/docs/es/operations/access_rights.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Derechos de acceso {#access-rights} Los usuarios y los derechos de acceso se configuran en la configuración del usuario. Esto suele ser `users.xml`. diff --git a/docs/es/operations/backup.md b/docs/es/operations/backup.md index 94eaa606052..5f1ffc79648 100644 --- a/docs/es/operations/backup.md +++ b/docs/es/operations/backup.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Copia de seguridad de datos {#data-backup} Mientras [replicación](table_engines/replication.md) proporciona protección contra fallas de hardware, no protege contra errores humanos: eliminación accidental de datos, eliminación de la tabla incorrecta o una tabla en el clúster incorrecto y errores de software que resultan en un procesamiento incorrecto de datos o daños en los datos. En muchos casos, errores como estos afectarán a todas las réplicas. ClickHouse tiene protecciones integradas para evitar algunos tipos de errores, por ejemplo, de forma predeterminada [no puede simplemente colocar tablas con un motor similar a MergeTree que contenga más de 50 Gb de datos](https://github.com/ClickHouse/ClickHouse/blob/v18.14.18-stable/dbms/programs/server/config.xml#L322-L330). Sin embargo, estas garantías no cubren todos los casos posibles y pueden eludirse. diff --git a/docs/es/operations/configuration_files.md b/docs/es/operations/configuration_files.md index 16df7afdaae..af508217469 100644 --- a/docs/es/operations/configuration_files.md +++ b/docs/es/operations/configuration_files.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/index.md b/docs/es/operations/index.md index aaa98a6bf05..78c89f786f3 100644 --- a/docs/es/operations/index.md +++ b/docs/es/operations/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Operación {#operations} El manual de operaciones de ClickHouse consta de las siguientes secciones principales: diff --git a/docs/es/operations/monitoring.md b/docs/es/operations/monitoring.md index 0584ed258fa..4729796dc83 100644 --- a/docs/es/operations/monitoring.md +++ b/docs/es/operations/monitoring.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Monitoreo {#monitoring} Usted puede monitorear: diff --git a/docs/es/operations/performance/sampling_query_profiler.md b/docs/es/operations/performance/sampling_query_profiler.md index 639dbfe972e..cf12c2b2e5a 100644 --- a/docs/es/operations/performance/sampling_query_profiler.md +++ b/docs/es/operations/performance/sampling_query_profiler.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/performance_test.md b/docs/es/operations/performance_test.md index 95bbef1be6c..62e5078691b 100644 --- a/docs/es/operations/performance_test.md +++ b/docs/es/operations/performance_test.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/quotas.md b/docs/es/operations/quotas.md index 89955f2b84a..4b53db1bbd0 100644 --- a/docs/es/operations/quotas.md +++ b/docs/es/operations/quotas.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Cuota {#quotas} Las cuotas le permiten limitar el uso de recursos durante un período de tiempo, o simplemente realizar un seguimiento del uso de recursos. diff --git a/docs/es/operations/requirements.md b/docs/es/operations/requirements.md index 43c21fa14a7..2aea6ba9a0c 100644 --- a/docs/es/operations/requirements.md +++ b/docs/es/operations/requirements.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Requisito {#requirements} ## CPU {#cpu} diff --git a/docs/es/operations/server_settings/index.md b/docs/es/operations/server_settings/index.md index 6fe199aaf89..f98b4c8b788 100644 --- a/docs/es/operations/server_settings/index.md +++ b/docs/es/operations/server_settings/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/server_settings/settings.md b/docs/es/operations/server_settings/settings.md index 73791250c5b..6c9c74592bd 100644 --- a/docs/es/operations/server_settings/settings.md +++ b/docs/es/operations/server_settings/settings.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Configuración del servidor {#server-settings} ## builtin\_dictionaries\_reload\_interval {#builtin-dictionaries-reload-interval} diff --git a/docs/es/operations/settings/constraints_on_settings.md b/docs/es/operations/settings/constraints_on_settings.md index 941fc8fc693..a67f6b38a54 100644 --- a/docs/es/operations/settings/constraints_on_settings.md +++ b/docs/es/operations/settings/constraints_on_settings.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/settings/index.md b/docs/es/operations/settings/index.md index 5fcfe99fd3d..a62525c786d 100644 --- a/docs/es/operations/settings/index.md +++ b/docs/es/operations/settings/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Configuración {#settings} Hay varias formas de realizar todos los ajustes que se describen a continuación. diff --git a/docs/es/operations/settings/permissions_for_queries.md b/docs/es/operations/settings/permissions_for_queries.md index 3aa50ba37d4..5f777aacf16 100644 --- a/docs/es/operations/settings/permissions_for_queries.md +++ b/docs/es/operations/settings/permissions_for_queries.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Permisos para consultas {#permissions_for_queries} Las consultas en ClickHouse se pueden dividir en varios tipos: diff --git a/docs/es/operations/settings/query_complexity.md b/docs/es/operations/settings/query_complexity.md index 452b3d62074..da39fd01a7b 100644 --- a/docs/es/operations/settings/query_complexity.md +++ b/docs/es/operations/settings/query_complexity.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Restricciones en la complejidad de consultas {#restrictions-on-query-complexity} Las restricciones en la complejidad de la consulta forman parte de la configuración. diff --git a/docs/es/operations/settings/settings.md b/docs/es/operations/settings/settings.md index 06a559b4c6e..a28ef316d83 100644 --- a/docs/es/operations/settings/settings.md +++ b/docs/es/operations/settings/settings.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Configuración {#settings} ## distributed\_product\_mode {#distributed-product-mode} diff --git a/docs/es/operations/settings/settings_profiles.md b/docs/es/operations/settings/settings_profiles.md index 056646844af..3bc93b7dc02 100644 --- a/docs/es/operations/settings/settings_profiles.md +++ b/docs/es/operations/settings/settings_profiles.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Perfiles de configuración {#settings-profiles} Un perfil de configuración es una colección de configuraciones agrupadas con el mismo nombre. Cada usuario de ClickHouse tiene un perfil. diff --git a/docs/es/operations/settings/settings_users.md b/docs/es/operations/settings/settings_users.md index 21c389332a3..32d3791b72f 100644 --- a/docs/es/operations/settings/settings_users.md +++ b/docs/es/operations/settings/settings_users.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/system_tables.md b/docs/es/operations/system_tables.md index 9fbb4176e28..af073bc0fcc 100644 --- a/docs/es/operations/system_tables.md +++ b/docs/es/operations/system_tables.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/aggregatingmergetree.md b/docs/es/operations/table_engines/aggregatingmergetree.md index 8cd485831da..9f49f3b76d3 100644 --- a/docs/es/operations/table_engines/aggregatingmergetree.md +++ b/docs/es/operations/table_engines/aggregatingmergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # AgregaciónMergeTree {#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. diff --git a/docs/es/operations/table_engines/buffer.md b/docs/es/operations/table_engines/buffer.md index a1545297ccf..f05101a485e 100644 --- a/docs/es/operations/table_engines/buffer.md +++ b/docs/es/operations/table_engines/buffer.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/collapsingmergetree.md b/docs/es/operations/table_engines/collapsingmergetree.md index 32ac0d64cf9..9a05e0effb7 100644 --- a/docs/es/operations/table_engines/collapsingmergetree.md +++ b/docs/es/operations/table_engines/collapsingmergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/custom_partitioning_key.md b/docs/es/operations/table_engines/custom_partitioning_key.md index 4d043716540..71cfeb52411 100644 --- a/docs/es/operations/table_engines/custom_partitioning_key.md +++ b/docs/es/operations/table_engines/custom_partitioning_key.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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](materializedview.md) basado en tablas MergeTree soporte de particionamiento, también. diff --git a/docs/es/operations/table_engines/dictionary.md b/docs/es/operations/table_engines/dictionary.md index 5ef899d078e..d9c76229517 100644 --- a/docs/es/operations/table_engines/dictionary.md +++ b/docs/es/operations/table_engines/dictionary.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Diccionario {#dictionary} El `Dictionary` el motor muestra el [Diccionario](../../query_language/dicts/external_dicts.md) datos como una tabla ClickHouse. diff --git a/docs/es/operations/table_engines/distributed.md b/docs/es/operations/table_engines/distributed.md index a4b20a4e74d..e47ceee011f 100644 --- a/docs/es/operations/table_engines/distributed.md +++ b/docs/es/operations/table_engines/distributed.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/external_data.md b/docs/es/operations/table_engines/external_data.md index a93c3bd3566..1a88dac6729 100644 --- a/docs/es/operations/table_engines/external_data.md +++ b/docs/es/operations/table_engines/external_data.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/operations/table_engines/file.md b/docs/es/operations/table_engines/file.md index 14f97ef910a..dbe9ee7d873 100644 --- a/docs/es/operations/table_engines/file.md +++ b/docs/es/operations/table_engines/file.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # File {#table_engines-file} El motor de tabla de archivos mantiene los datos en un archivo en uno de los [file diff --git a/docs/es/operations/table_engines/generate.md b/docs/es/operations/table_engines/generate.md index b9f027a6162..11993bcc353 100644 --- a/docs/es/operations/table_engines/generate.md +++ b/docs/es/operations/table_engines/generate.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # GenerateRandom {#table_engines-generate} El motor de tabla GenerateRandom produce datos aleatorios para el esquema de tabla determinado. diff --git a/docs/es/operations/table_engines/graphitemergetree.md b/docs/es/operations/table_engines/graphitemergetree.md index 75c27347328..3db9f224807 100644 --- a/docs/es/operations/table_engines/graphitemergetree.md +++ b/docs/es/operations/table_engines/graphitemergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/hdfs.md b/docs/es/operations/table_engines/hdfs.md index 4ca0292b3bd..48078611bb0 100644 --- a/docs/es/operations/table_engines/hdfs.md +++ b/docs/es/operations/table_engines/hdfs.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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 diff --git a/docs/es/operations/table_engines/index.md b/docs/es/operations/table_engines/index.md index f43911874bc..de5a13de359 100644 --- a/docs/es/operations/table_engines/index.md +++ b/docs/es/operations/table_engines/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Motores de mesa {#table_engines} El motor de tabla (tipo de tabla) determina: diff --git a/docs/es/operations/table_engines/jdbc.md b/docs/es/operations/table_engines/jdbc.md index eacefe25701..7d6d5649df6 100644 --- a/docs/es/operations/table_engines/jdbc.md +++ b/docs/es/operations/table_engines/jdbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/operations/table_engines/join.md b/docs/es/operations/table_engines/join.md index 4a21e7f8d6c..65ec4b950a6 100644 --- a/docs/es/operations/table_engines/join.md +++ b/docs/es/operations/table_engines/join.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Unir {#join} Estructura de datos preparada para usar en [UNIR](../../query_language/select.md#select-join) operación. diff --git a/docs/es/operations/table_engines/kafka.md b/docs/es/operations/table_engines/kafka.md index 54b211c3115..40bd3ce2578 100644 --- a/docs/es/operations/table_engines/kafka.md +++ b/docs/es/operations/table_engines/kafka.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Kafka {#kafka} Este motor funciona con [Acerca de nosotros](http://kafka.apache.org/). diff --git a/docs/es/operations/table_engines/log.md b/docs/es/operations/table_engines/log.md index 8e029f45408..595e2532d45 100644 --- a/docs/es/operations/table_engines/log.md +++ b/docs/es/operations/table_engines/log.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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](log_family.md) artículo. diff --git a/docs/es/operations/table_engines/log_family.md b/docs/es/operations/table_engines/log_family.md index 0f9d77ee062..1a2b3f2bb70 100644 --- a/docs/es/operations/table_engines/log_family.md +++ b/docs/es/operations/table_engines/log_family.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/materializedview.md b/docs/es/operations/table_engines/materializedview.md index 4e12075c542..d40213a4d18 100644 --- a/docs/es/operations/table_engines/materializedview.md +++ b/docs/es/operations/table_engines/materializedview.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Método de codificación de datos: {#materializedview} Se utiliza para implementar vistas materializadas (para obtener más información, consulte [CREAR TABLA](../../query_language/create.md)). Para almacenar datos, utiliza un motor diferente que se especificó al crear la vista. Al leer desde una tabla, solo usa este motor. diff --git a/docs/es/operations/table_engines/memory.md b/docs/es/operations/table_engines/memory.md index 333d74ccd9b..66b1a79c4e4 100644 --- a/docs/es/operations/table_engines/memory.md +++ b/docs/es/operations/table_engines/memory.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/merge.md b/docs/es/operations/table_engines/merge.md index 33789b56570..7147a55b975 100644 --- a/docs/es/operations/table_engines/merge.md +++ b/docs/es/operations/table_engines/merge.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/mergetree.md b/docs/es/operations/table_engines/mergetree.md index 53f16e3904e..53d89427ee0 100644 --- a/docs/es/operations/table_engines/mergetree.md +++ b/docs/es/operations/table_engines/mergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/mysql.md b/docs/es/operations/table_engines/mysql.md index 3ce17a02837..bb7d3016221 100644 --- a/docs/es/operations/table_engines/mysql.md +++ b/docs/es/operations/table_engines/mysql.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # MySQL {#mysql} El motor MySQL le permite realizar `SELECT` Consultas sobre datos almacenados en un servidor MySQL remoto. diff --git a/docs/es/operations/table_engines/null.md b/docs/es/operations/table_engines/null.md index 39bec0533f5..7915af8e108 100644 --- a/docs/es/operations/table_engines/null.md +++ b/docs/es/operations/table_engines/null.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nulo {#null} Al escribir en una tabla Null, los datos se ignoran. Al leer desde una tabla Null, la respuesta está vacía. diff --git a/docs/es/operations/table_engines/odbc.md b/docs/es/operations/table_engines/odbc.md index d5bf8fc8492..765d6b814aa 100644 --- a/docs/es/operations/table_engines/odbc.md +++ b/docs/es/operations/table_engines/odbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/operations/table_engines/replacingmergetree.md b/docs/es/operations/table_engines/replacingmergetree.md index 7b59f08223f..149c0211662 100644 --- a/docs/es/operations/table_engines/replacingmergetree.md +++ b/docs/es/operations/table_engines/replacingmergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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) de valor). diff --git a/docs/es/operations/table_engines/replication.md b/docs/es/operations/table_engines/replication.md index 143a7aa418f..36f0b877b81 100644 --- a/docs/es/operations/table_engines/replication.md +++ b/docs/es/operations/table_engines/replication.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Replicación de datos {#table_engines-replication} La replicación solo se admite para tablas de la familia MergeTree: diff --git a/docs/es/operations/table_engines/set.md b/docs/es/operations/table_engines/set.md index c56ab0ab96a..4c21397db0f 100644 --- a/docs/es/operations/table_engines/set.md +++ b/docs/es/operations/table_engines/set.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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”). diff --git a/docs/es/operations/table_engines/stripelog.md b/docs/es/operations/table_engines/stripelog.md index b4b32672cdb..a6131ae5d32 100644 --- a/docs/es/operations/table_engines/stripelog.md +++ b/docs/es/operations/table_engines/stripelog.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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](log_family.md) artículo. diff --git a/docs/es/operations/table_engines/summingmergetree.md b/docs/es/operations/table_engines/summingmergetree.md index 2b2fedfe602..dae6f366217 100644 --- a/docs/es/operations/table_engines/summingmergetree.md +++ b/docs/es/operations/table_engines/summingmergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/table_engines/tinylog.md b/docs/es/operations/table_engines/tinylog.md index a563d7a523e..1d9ac4c3b73 100644 --- a/docs/es/operations/table_engines/tinylog.md +++ b/docs/es/operations/table_engines/tinylog.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # TinyLog {#tinylog} El motor pertenece a la familia de motores de registro. Ver [Familia del motor de registro](log_family.md) para las propiedades comunes de los motores de registro y sus diferencias. diff --git a/docs/es/operations/table_engines/url.md b/docs/es/operations/table_engines/url.md index 5ccc690670f..d606de56083 100644 --- a/docs/es/operations/table_engines/url.md +++ b/docs/es/operations/table_engines/url.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nombre de la red inalámbrica (SSID):) {#table_engines-url} Administra datos en un servidor HTTP/HTTPS remoto. Este motor es similar diff --git a/docs/es/operations/table_engines/versionedcollapsingmergetree.md b/docs/es/operations/table_engines/versionedcollapsingmergetree.md index 99a0df0c493..c12c27cef88 100644 --- a/docs/es/operations/table_engines/versionedcollapsingmergetree.md +++ b/docs/es/operations/table_engines/versionedcollapsingmergetree.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # VersionedCollapsingMergeTree {#versionedcollapsingmergetree} Este motor: diff --git a/docs/es/operations/table_engines/view.md b/docs/es/operations/table_engines/view.md index 3fc432ef249..9ff0ec1836e 100644 --- a/docs/es/operations/table_engines/view.md +++ b/docs/es/operations/table_engines/view.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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). diff --git a/docs/es/operations/tips.md b/docs/es/operations/tips.md index 3913a6b1909..e07c29a5e5a 100644 --- a/docs/es/operations/tips.md +++ b/docs/es/operations/tips.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Recomendaciones de uso {#usage-recommendations} ## CPU Scaling Governor {#cpu-scaling-governor} diff --git a/docs/es/operations/troubleshooting.md b/docs/es/operations/troubleshooting.md index 4996ee2bebc..d381ef8ffcf 100644 --- a/docs/es/operations/troubleshooting.md +++ b/docs/es/operations/troubleshooting.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Solución de problemas {#troubleshooting} - [Instalación](#troubleshooting-installation-errors) diff --git a/docs/es/operations/update.md b/docs/es/operations/update.md index d03f6069426..ce93dab08dc 100644 --- a/docs/es/operations/update.md +++ b/docs/es/operations/update.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Actualización de ClickHouse {#clickhouse-update} Si se instaló ClickHouse desde paquetes deb, ejecute los siguientes comandos en el servidor: diff --git a/docs/es/operations/utils/clickhouse-benchmark.md b/docs/es/operations/utils/clickhouse-benchmark.md index f63dc41c67e..9c91d378b98 100644 --- a/docs/es/operations/utils/clickhouse-benchmark.md +++ b/docs/es/operations/utils/clickhouse-benchmark.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Sistema abierto. {#clickhouse-benchmark} Se conecta a un servidor ClickHouse y envía repetidamente las consultas especificadas. diff --git a/docs/es/operations/utils/clickhouse-copier.md b/docs/es/operations/utils/clickhouse-copier.md index c398301998a..caf282c3806 100644 --- a/docs/es/operations/utils/clickhouse-copier.md +++ b/docs/es/operations/utils/clickhouse-copier.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/operations/utils/clickhouse-local.md b/docs/es/operations/utils/clickhouse-local.md index 0a4cc067170..faad161ba69 100644 --- a/docs/es/operations/utils/clickhouse-local.md +++ b/docs/es/operations/utils/clickhouse-local.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Sistema abierto. {#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. diff --git a/docs/es/operations/utils/index.md b/docs/es/operations/utils/index.md index b7bee1def34..2911d46f310 100644 --- a/docs/es/operations/utils/index.md +++ b/docs/es/operations/utils/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Utilidad ClickHouse {#clickhouse-utility} - [Sistema abierto.](clickhouse-local.md) — Permite ejecutar consultas SQL en datos sin detener el servidor ClickHouse, similar a cómo `awk` hace esto. diff --git a/docs/es/query_language/agg_functions/combinators.md b/docs/es/query_language/agg_functions/combinators.md index 59216a8aa78..3abe9eb0dcb 100644 --- a/docs/es/query_language/agg_functions/combinators.md +++ b/docs/es/query_language/agg_functions/combinators.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/query_language/agg_functions/index.md b/docs/es/query_language/agg_functions/index.md index f230aa00439..297cefd8d2d 100644 --- a/docs/es/query_language/agg_functions/index.md +++ b/docs/es/query_language/agg_functions/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # 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. diff --git a/docs/es/query_language/agg_functions/parametric_functions.md b/docs/es/query_language/agg_functions/parametric_functions.md index 82806bf2636..70703657548 100644 --- a/docs/es/query_language/agg_functions/parametric_functions.md +++ b/docs/es/query_language/agg_functions/parametric_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones agregadas paramétricas {#aggregate_functions_parametric} Algunas funciones agregadas pueden aceptar no solo columnas de argumentos (utilizadas para la compresión), sino un conjunto de parámetros: constantes para la inicialización. La sintaxis es de dos pares de corchetes en lugar de uno. El primero es para parámetros, y el segundo es para argumentos. diff --git a/docs/es/query_language/agg_functions/reference.md b/docs/es/query_language/agg_functions/reference.md deleted file mode 120000 index c5651cb0793..00000000000 --- a/docs/es/query_language/agg_functions/reference.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/query_language/agg_functions/reference.md \ No newline at end of file diff --git a/docs/es/query_language/agg_functions/reference.md b/docs/es/query_language/agg_functions/reference.md new file mode 100644 index 00000000000..467d50ae93f --- /dev/null +++ b/docs/es/query_language/agg_functions/reference.md @@ -0,0 +1,1834 @@ +--- +machine_translated: true +--- + +# Referencia de la función {#function-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](../../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](../../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](../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. + +## sumMap(clave, valor) {#agg_functions-summap} + +Totals el ‘value’ matriz de acuerdo con las claves especificadas en el ‘key’ matriz. +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 + ) +) ENGINE = Log; +INSERT INTO sum_map VALUES + ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]), + ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]), + ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]), + ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]); +SELECT + timeslot, + sumMap(statusMap.status, statusMap.requests) +FROM sum_map +GROUP BY timeslot +``` + +``` text +┌────────────timeslot─┬─sumMap(statusMap.status, statusMap.requests)─┐ +│ 2000-01-01 00:00:00 │ ([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]) │ +└─────────────────────┴──────────────────────────────────────────────┘ +``` + +## 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](../../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](../../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](../../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](../../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 +``` + +## Para obtener más información, consulta nuestra Política de privacidad y nuestras Condiciones de uso) {#agg-function-timeseriesgroupsum} + +`timeSeriesGroupSum` puede agregar diferentes series de tiempo que muestran la marca de tiempo no la alineación. +Utilizará la interpolación lineal entre dos marcas de tiempo de muestra y luego sumará series temporales juntas. + +- `uid` es la identificación única de la serie temporal, `UInt64`. +- `timestamp` es el tipo Int64 para admitir milisegundos o microsegundos. +- `value` es la métrica. + +La función devuelve una matriz de tuplas con `(timestamp, aggregated_value)` par. + +Antes de utilizar esta función, asegúrese de `timestamp` está en orden ascendente. + +Ejemplo: + +``` text +┌─uid─┬─timestamp─┬─value─┐ +│ 1 │ 2 │ 0.2 │ +│ 1 │ 7 │ 0.7 │ +│ 1 │ 12 │ 1.2 │ +│ 1 │ 17 │ 1.7 │ +│ 1 │ 25 │ 2.5 │ +│ 2 │ 3 │ 0.6 │ +│ 2 │ 8 │ 1.6 │ +│ 2 │ 12 │ 2.4 │ +│ 2 │ 18 │ 3.6 │ +│ 2 │ 24 │ 4.8 │ +└─────┴───────────┴───────┘ +``` + +``` sql +CREATE TABLE time_series( + uid UInt64, + timestamp Int64, + value Float64 +) ENGINE = Memory; +INSERT INTO time_series VALUES + (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), + (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); + +SELECT timeSeriesGroupSum(uid, timestamp, value) +FROM ( + SELECT * FROM time_series order by timestamp ASC +); +``` + +Y el resultado será: + +``` text +[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] +``` + +## También puede utilizar el siguiente ejemplo:) {#agg-function-timeseriesgroupratesum} + +Del mismo modo timeSeriesGroupRateSum, timeSeriesGroupRateSum calculará la tasa de series temporales y luego sumará las tasas juntas. +Además, la marca de tiempo debe estar en orden ascendente antes de usar esta función. + +Use esta función, el resultado anterior será: + +``` text +[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] +``` + +## Acerca de) {#agg_function-avg} + +Calcula el promedio. +Solo funciona para números. +El resultado es siempre Float64. + +## 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](../../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](../../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](../../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`. + +## Para obtener más información, consulte el siguiente enlace:) {#grouparrayinsertatvalue-position} + +Inserta un valor en la matriz en la posición especificada. + +!!! note "Nota" + Esta función utiliza posiciones de base cero, contrariamente a las posiciones de base única convencionales para matrices SQL. + +Accepts the value and position as input. If several values ​​are inserted into the same position, any of them might end up in the resulting array (the first one will be used in the case of single-threaded execution). If no value is inserted into a position, the position is assigned the default value. + +Parámetros opcionales: + +- El valor predeterminado para sustituir en posiciones vacías. +- La longitud de la matriz resultante. Esto le permite recibir matrices del mismo tamaño para todas las claves agregadas. Al utilizar este parámetro, se debe especificar el valor predeterminado. + +## 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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../data_types/datetime.md). + +**Valor devuelto** + +- Cuantil aproximado del nivel especificado. + +Tipo: + +- [Float64](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../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](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../data_types/datetime.md). + +**Valor devuelto** + +- Cuantil del nivel especificado. + +Tipo: + +- [Float64](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../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](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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\*](../../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](../../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](../select.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\*](../../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](../../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](../select.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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../data_types/datetime.md). + +**Valor devuelto** + +- Cuantil aproximado del nivel especificado. + +Tipo: + +- [Float64](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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](../../data_types/index.md#data_types), [Fecha](../../data_types/date.md) o [FechaHora](../../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](../../data_types/float.md) para la entrada de tipo de datos numéricos. +- [Fecha](../../data_types/date.md) si los valores de entrada tienen `Date` tipo. +- [FechaHora](../../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 `+∞`. + +## 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`. + +## Soporte técnico) {#stddevsampx} + +El resultado es igual a la raíz cuadrada de `varSamp(x)`. + +## stddevPop(x) {#stddevpopx} + +El resultado es igual a la raíz cuadrada de `varPop(x)`. + +## 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](../../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 +∞. + +## covarPop(x, y) {#covarpopx-y} + +Calcula el valor de `Σ((x - x̅)(y - y̅)) / n`. + +## corr(x, y) {#corrx-y} + +Calcula el coeficiente de correlación de Pearson: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`. + +## 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](../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](../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](../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/query_language/alter.md b/docs/es/query_language/alter.md index e4726159bd2..31342199735 100644 --- a/docs/es/query_language/alter.md +++ b/docs/es/query_language/alter.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ## ALTERAR {#query_language_queries_alter} El `ALTER` consulta sólo se admite para `*MergeTree` mesas, así como `Merge`y`Distributed`. La consulta tiene varias variaciones. diff --git a/docs/es/query_language/create.md b/docs/es/query_language/create.md index d366001e261..ba6f5f1849b 100644 --- a/docs/es/query_language/create.md +++ b/docs/es/query_language/create.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # CREAR Consultas {#create-queries} ## CREAR BASE DE DATOS {#query-language-create-database} diff --git a/docs/es/query_language/dicts/external_dicts.md b/docs/es/query_language/dicts/external_dicts.md index 28e408b57e2..e0b09c6015e 100644 --- a/docs/es/query_language/dicts/external_dicts.md +++ b/docs/es/query_language/dicts/external_dicts.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Diccionarios externos {#dicts-external-dicts} Puede agregar sus propios diccionarios de varias fuentes de datos. El origen de datos de un diccionario puede ser un archivo ejecutable o de texto local, un recurso HTTP u otro DBMS. Para obtener más información, consulte “[Fuentes para diccionarios externos](external_dicts_dict_sources.md)”. diff --git a/docs/es/query_language/dicts/external_dicts_dict.md b/docs/es/query_language/dicts/external_dicts_dict.md index 1614a78fb6b..1d1b7bbcb0e 100644 --- a/docs/es/query_language/dicts/external_dicts_dict.md +++ b/docs/es/query_language/dicts/external_dicts_dict.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Configuración de un diccionario externo {#dicts-external-dicts-dict} Si el diccionario se configura usando un archivo xml, la configuración del diccionario tiene la siguiente estructura: diff --git a/docs/es/query_language/dicts/external_dicts_dict_hierarchical.md b/docs/es/query_language/dicts/external_dicts_dict_hierarchical.md index 095a3cb2b78..2665fc76d0e 100644 --- a/docs/es/query_language/dicts/external_dicts_dict_hierarchical.md +++ b/docs/es/query_language/dicts/external_dicts_dict_hierarchical.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Diccionarios jerárquicos {#hierarchical-dictionaries} ClickHouse soporta diccionarios jerárquicos con un [llave numérica](external_dicts_dict_structure.md#ext_dict-numeric-key). diff --git a/docs/es/query_language/dicts/external_dicts_dict_layout.md b/docs/es/query_language/dicts/external_dicts_dict_layout.md index 7467a5d8100..cfa7dcaf304 100644 --- a/docs/es/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/es/query_language/dicts/external_dicts_dict_layout.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Almacenamiento de diccionarios en la memoria {#dicts-external-dicts-dict-layout} Hay una variad de formas de almacenar diccionarios en la memoria. diff --git a/docs/es/query_language/dicts/external_dicts_dict_lifetime.md b/docs/es/query_language/dicts/external_dicts_dict_lifetime.md index cbea58f8dd4..4e004d0ee7f 100644 --- a/docs/es/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/es/query_language/dicts/external_dicts_dict_lifetime.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Actualizaciones del diccionario {#dictionary-updates} ClickHouse actualiza periódicamente los diccionarios. El intervalo de actualización para los diccionarios completamente descargados y el intervalo de invalidación para los diccionarios almacenados en caché se `` etiqueta en segundos. diff --git a/docs/es/query_language/dicts/external_dicts_dict_sources.md b/docs/es/query_language/dicts/external_dicts_dict_sources.md index c9d4804a300..953e0d193e0 100644 --- a/docs/es/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/es/query_language/dicts/external_dicts_dict_sources.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Fuentes de diccionarios externos {#dicts-external-dicts-dict-sources} Un diccionario externo se puede conectar desde muchas fuentes diferentes. diff --git a/docs/es/query_language/dicts/external_dicts_dict_structure.md b/docs/es/query_language/dicts/external_dicts_dict_structure.md index a337a8b71e2..4b97e3e1701 100644 --- a/docs/es/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/es/query_language/dicts/external_dicts_dict_structure.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Clave y campos del diccionario {#dictionary-key-and-fields} El `` cláusula describe la clave del diccionario y los campos disponibles para consultas. diff --git a/docs/es/query_language/dicts/index.md b/docs/es/query_language/dicts/index.md index 77e6f0db41c..6007869a9b1 100644 --- a/docs/es/query_language/dicts/index.md +++ b/docs/es/query_language/dicts/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Diccionario {#dictionaries} Un diccionario es un mapeo (`key -> attributes`) que es conveniente para varios tipos de listas de referencia. diff --git a/docs/es/query_language/dicts/internal_dicts.md b/docs/es/query_language/dicts/internal_dicts.md index 60571371c02..4795975b1cb 100644 --- a/docs/es/query_language/dicts/internal_dicts.md +++ b/docs/es/query_language/dicts/internal_dicts.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Diccionarios internos {#internal_dicts} ClickHouse contiene una función integrada para trabajar con una geobase. diff --git a/docs/es/query_language/functions/arithmetic_functions.md b/docs/es/query_language/functions/arithmetic_functions.md index c41d52a4991..9ec7995edad 100644 --- a/docs/es/query_language/functions/arithmetic_functions.md +++ b/docs/es/query_language/functions/arithmetic_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones aritméticas {#arithmetic-functions} Para todas las funciones aritméticas, el tipo de resultado se calcula como el tipo de número más pequeño en el que encaja el resultado, si existe dicho tipo. El mínimo se toma simultáneamente en función del número de bits, si está firmado y si flota. Si no hay suficientes bits, se toma el tipo de bit más alto. diff --git a/docs/es/query_language/functions/array_functions.md b/docs/es/query_language/functions/array_functions.md index b42cf192277..e1dd2a38f69 100644 --- a/docs/es/query_language/functions/array_functions.md +++ b/docs/es/query_language/functions/array_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con matrices {#functions-for-working-with-arrays} ## vaciar {#function-empty} diff --git a/docs/es/query_language/functions/array_join.md b/docs/es/query_language/functions/array_join.md index b9df26ec71f..25645efc45c 100644 --- a/docs/es/query_language/functions/array_join.md +++ b/docs/es/query_language/functions/array_join.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # arrayJoin función {#functions_arrayjoin} Esta es una función muy inusual. diff --git a/docs/es/query_language/functions/bit_functions.md b/docs/es/query_language/functions/bit_functions.md index d74c89c3633..309c55e33ca 100644 --- a/docs/es/query_language/functions/bit_functions.md +++ b/docs/es/query_language/functions/bit_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Bit funciones {#bit-functions} Las funciones de bits funcionan para cualquier par de tipos de UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32 o Float64. diff --git a/docs/es/query_language/functions/bitmap_functions.md b/docs/es/query_language/functions/bitmap_functions.md index a16ff2aad87..65fcc33731a 100644 --- a/docs/es/query_language/functions/bitmap_functions.md +++ b/docs/es/query_language/functions/bitmap_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de mapa de bits {#bitmap-functions} Las funciones de mapa de bits funcionan para dos mapas de bits Cálculo del valor del objeto, es devolver un nuevo mapa de bits o cardinalidad mientras se usa el cálculo de la fórmula, como y, o, xor, y no, etc. diff --git a/docs/es/query_language/functions/comparison_functions.md b/docs/es/query_language/functions/comparison_functions.md index c64cd659bb9..ebd799508a9 100644 --- a/docs/es/query_language/functions/comparison_functions.md +++ b/docs/es/query_language/functions/comparison_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de comparación {#comparison-functions} Las funciones de comparación siempre devuelven 0 o 1 (Uint8). diff --git a/docs/es/query_language/functions/conditional_functions.md b/docs/es/query_language/functions/conditional_functions.md index efed413827b..9506ea2aaa2 100644 --- a/docs/es/query_language/functions/conditional_functions.md +++ b/docs/es/query_language/functions/conditional_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones condicionales {#conditional-functions} ## si {#if} diff --git a/docs/es/query_language/functions/date_time_functions.md b/docs/es/query_language/functions/date_time_functions.md index a9e651b9e3a..d1f2d2622cd 100644 --- a/docs/es/query_language/functions/date_time_functions.md +++ b/docs/es/query_language/functions/date_time_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con fechas y horas {#functions-for-working-with-dates-and-times} Soporte para zonas horarias diff --git a/docs/es/query_language/functions/encoding_functions.md b/docs/es/query_language/functions/encoding_functions.md index e8b59c85588..dadb4b4db00 100644 --- a/docs/es/query_language/functions/encoding_functions.md +++ b/docs/es/query_language/functions/encoding_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de codificación {#encoding-functions} ## char {#char} diff --git a/docs/es/query_language/functions/ext_dict_functions.md b/docs/es/query_language/functions/ext_dict_functions.md index 86a37cef3ce..fe7515b8846 100644 --- a/docs/es/query_language/functions/ext_dict_functions.md +++ b/docs/es/query_language/functions/ext_dict_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con diccionarios externos {#ext_dict_functions} Para obtener información sobre cómo conectar y configurar diccionarios externos, consulte [Diccionarios externos](../dicts/external_dicts.md). diff --git a/docs/es/query_language/functions/functions_for_nulls.md b/docs/es/query_language/functions/functions_for_nulls.md index f3c71fc7b89..e343667ce6e 100644 --- a/docs/es/query_language/functions/functions_for_nulls.md +++ b/docs/es/query_language/functions/functions_for_nulls.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con agregados anulables {#functions-for-working-with-nullable-aggregates} ## IsNull {#isnull} diff --git a/docs/es/query_language/functions/geo.md b/docs/es/query_language/functions/geo.md index 62083a6a09e..eba818b5639 100644 --- a/docs/es/query_language/functions/geo.md +++ b/docs/es/query_language/functions/geo.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con coordenadas geográficas {#functions-for-working-with-geographical-coordinates} ## GranCircleDistance {#greatcircledistance} diff --git a/docs/es/query_language/functions/hash_functions.md b/docs/es/query_language/functions/hash_functions.md index b1e63417ef1..0bb985c2af0 100644 --- a/docs/es/query_language/functions/hash_functions.md +++ b/docs/es/query_language/functions/hash_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones hash {#hash-functions} Las funciones Hash se pueden usar para la barajada pseudoaleatoria determinista de elementos. diff --git a/docs/es/query_language/functions/higher_order_functions.md b/docs/es/query_language/functions/higher_order_functions.md index c867c5dddf0..fce1c63d2c1 100644 --- a/docs/es/query_language/functions/higher_order_functions.md +++ b/docs/es/query_language/functions/higher_order_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de orden superior {#higher-order-functions} ## `->` operador, función lambda (params, expr) {#operator-lambdaparams-expr-function} diff --git a/docs/es/query_language/functions/in_functions.md b/docs/es/query_language/functions/in_functions.md index a0ebd07a3ea..9ecd8c1f8ad 100644 --- a/docs/es/query_language/functions/in_functions.md +++ b/docs/es/query_language/functions/in_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para implementar el operador IN {#functions-for-implementing-the-in-operator} ## Información de uso {#in-functions} diff --git a/docs/es/query_language/functions/index.md b/docs/es/query_language/functions/index.md index 8e69d670b59..80d528393ea 100644 --- a/docs/es/query_language/functions/index.md +++ b/docs/es/query_language/functions/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Función {#functions} Hay al menos \* dos tipos de funciones: funciones regulares (simplemente se llaman “functions”) y funciones agregadas. Estos son conceptos completamente diferentes. Las funciones regulares funcionan como si se aplicaran a cada fila por separado (para cada fila, el resultado de la función no depende de las otras filas). Las funciones agregadas acumulan un conjunto de valores de varias filas (es decir,dependen de todo el conjunto de filas). diff --git a/docs/es/query_language/functions/introspection.md b/docs/es/query_language/functions/introspection.md index dd0fb0c61f5..634cf52118c 100644 --- a/docs/es/query_language/functions/introspection.md +++ b/docs/es/query_language/functions/introspection.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de introspección {#introspection-functions} Puede utilizar las funciones descritas en este capítulo para [ELFO](https://en.wikipedia.org/wiki/Executable_and_Linkable_Format) y [ENANO](https://en.wikipedia.org/wiki/DWARF) para la creación de perfiles de consultas. diff --git a/docs/es/query_language/functions/ip_address_functions.md b/docs/es/query_language/functions/ip_address_functions.md index 3705168a6e2..5f55fc55fab 100644 --- a/docs/es/query_language/functions/ip_address_functions.md +++ b/docs/es/query_language/functions/ip_address_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con direcciones IP {#functions-for-working-with-ip-addresses} ## Número de código: IPv4NumToString (num) {#ipv4numtostringnum} diff --git a/docs/es/query_language/functions/json_functions.md b/docs/es/query_language/functions/json_functions.md index 10c6b4b3d6f..d8732b281e1 100644 --- a/docs/es/query_language/functions/json_functions.md +++ b/docs/es/query_language/functions/json_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con JSON {#functions-for-working-with-json} En el Yandex.Metrica, JSON es transmitido por los usuarios como parámetros de sesión. Hay algunas funciones especiales para trabajar con este JSON. (Aunque en la mayoría de los casos, los JSON también se procesan previamente, y los valores resultantes se colocan en columnas separadas en su formato procesado.) Todas estas funciones se basan en sólidas suposiciones sobre lo que puede ser el JSON, pero tratan de hacer lo menos posible para hacer el trabajo. diff --git a/docs/es/query_language/functions/logical_functions.md b/docs/es/query_language/functions/logical_functions.md index ecfb4c51f30..5de5607f134 100644 --- a/docs/es/query_language/functions/logical_functions.md +++ b/docs/es/query_language/functions/logical_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones lógicas {#logical-functions} Las funciones lógicas aceptan cualquier tipo numérico, pero devuelven un número UInt8 igual a 0 o 1. diff --git a/docs/es/query_language/functions/machine_learning_functions.md b/docs/es/query_language/functions/machine_learning_functions.md index e205c91cc00..b1c80ee7ce0 100644 --- a/docs/es/query_language/functions/machine_learning_functions.md +++ b/docs/es/query_language/functions/machine_learning_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de aprendizaje automático {#machine-learning-functions} ## evalMLMethod (predicción) {#machine_learning_methods-evalmlmethod} diff --git a/docs/es/query_language/functions/math_functions.md b/docs/es/query_language/functions/math_functions.md index 77ab4dff4c1..1cc15ba944d 100644 --- a/docs/es/query_language/functions/math_functions.md +++ b/docs/es/query_language/functions/math_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones matemáticas {#mathematical-functions} Todas las funciones devuelven un número Float64. La precisión del resultado es cercana a la precisión máxima posible, pero el resultado puede no coincidir con el número representable de la máquina más cercano al número real correspondiente. diff --git a/docs/es/query_language/functions/other_functions.md b/docs/es/query_language/functions/other_functions.md index 5879ac4ad3c..8201620f809 100644 --- a/docs/es/query_language/functions/other_functions.md +++ b/docs/es/query_language/functions/other_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Otras funciones {#other-functions} ## nombre de host() {#hostname} diff --git a/docs/es/query_language/functions/random_functions.md b/docs/es/query_language/functions/random_functions.md index fd9496bec03..19547210aa0 100644 --- a/docs/es/query_language/functions/random_functions.md +++ b/docs/es/query_language/functions/random_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para generar números pseudoaleatorios {#functions-for-generating-pseudo-random-numbers} Se utilizan generadores no criptográficos de números pseudoaleatorios. diff --git a/docs/es/query_language/functions/rounding_functions.md b/docs/es/query_language/functions/rounding_functions.md index 7959dfa15bd..7d9eb229586 100644 --- a/docs/es/query_language/functions/rounding_functions.md +++ b/docs/es/query_language/functions/rounding_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de redondeo {#rounding-functions} ## Piso(x\[, N\]) {#floorx-n} diff --git a/docs/es/query_language/functions/splitting_merging_functions.md b/docs/es/query_language/functions/splitting_merging_functions.md index 37c76748e2f..9f257fe2f61 100644 --- a/docs/es/query_language/functions/splitting_merging_functions.md +++ b/docs/es/query_language/functions/splitting_merging_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para dividir y fusionar cadenas y matrices {#functions-for-splitting-and-merging-strings-and-arrays} ## Por ejemplo:) {#splitbycharseparator-s} diff --git a/docs/es/query_language/functions/string_functions.md b/docs/es/query_language/functions/string_functions.md index 8e756452aed..bd25c9128db 100644 --- a/docs/es/query_language/functions/string_functions.md +++ b/docs/es/query_language/functions/string_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con cadenas {#functions-for-working-with-strings} ## Vaciar {#empty} diff --git a/docs/es/query_language/functions/string_replace_functions.md b/docs/es/query_language/functions/string_replace_functions.md index 66bb57e70a0..3449162c5c5 100644 --- a/docs/es/query_language/functions/string_replace_functions.md +++ b/docs/es/query_language/functions/string_replace_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para buscar y reemplazar en cadenas {#functions-for-searching-and-replacing-in-strings} ## replaceOne(pajar, patrón, reemplazo) {#replaceonehaystack-pattern-replacement} diff --git a/docs/es/query_language/functions/string_search_functions.md b/docs/es/query_language/functions/string_search_functions.md index b4cef0216ab..c9ef20be853 100644 --- a/docs/es/query_language/functions/string_search_functions.md +++ b/docs/es/query_language/functions/string_search_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para buscar cadenas {#functions-for-searching-strings} La búsqueda distingue entre mayúsculas y minúsculas de forma predeterminada en todas estas funciones. Hay variantes separadas para la búsqueda insensible a mayúsculas y minúsculas. diff --git a/docs/es/query_language/functions/type_conversion_functions.md b/docs/es/query_language/functions/type_conversion_functions.md index 206695930af..9d6751a7c10 100644 --- a/docs/es/query_language/functions/type_conversion_functions.md +++ b/docs/es/query_language/functions/type_conversion_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de conversión de tipos {#type-conversion-functions} ## Problemas comunes de conversiones numéricas {#numeric-conversion-issues} diff --git a/docs/es/query_language/functions/url_functions.md b/docs/es/query_language/functions/url_functions.md index 7601f357e92..5cfb751446b 100644 --- a/docs/es/query_language/functions/url_functions.md +++ b/docs/es/query_language/functions/url_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con URL {#functions-for-working-with-urls} Todas estas funciones no siguen el RFC. Se simplifican al máximo para mejorar el rendimiento. diff --git a/docs/es/query_language/functions/uuid_functions.md b/docs/es/query_language/functions/uuid_functions.md index 469e4dd807c..b83411c08ec 100644 --- a/docs/es/query_language/functions/uuid_functions.md +++ b/docs/es/query_language/functions/uuid_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con UUID {#functions-for-working-with-uuid} Las funciones para trabajar con UUID se enumeran a continuación. diff --git a/docs/es/query_language/functions/ym_dict_functions.md b/docs/es/query_language/functions/ym_dict_functions.md index f80204f777f..7e5d7b2d73e 100644 --- a/docs/es/query_language/functions/ym_dict_functions.md +++ b/docs/es/query_language/functions/ym_dict_functions.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones para trabajar con Yandex.Métrica, diccionarios {#functions-for-working-with-yandex-metrica-dictionaries} Para que las funciones a continuación funcionen, la configuración del servidor debe especificar las rutas y direcciones para obtener todo el Yandex.Diccionarios Metrica. Los diccionarios se cargan en la primera llamada de cualquiera de estas funciones. Si no se pueden cargar las listas de referencia, se produce una excepción. diff --git a/docs/es/query_language/index.md b/docs/es/query_language/index.md index a7f21a41441..fa981365e64 100644 --- a/docs/es/query_language/index.md +++ b/docs/es/query_language/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Referencia SQL {#sql-reference} - [SELECCIONAR](select.md) diff --git a/docs/es/query_language/insert_into.md b/docs/es/query_language/insert_into.md index 3decb6fdd8b..62a0f6fdaf2 100644 --- a/docs/es/query_language/insert_into.md +++ b/docs/es/query_language/insert_into.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ## INSERTAR {#insert} Adición de datos. diff --git a/docs/es/query_language/misc.md b/docs/es/query_language/misc.md index 747f0852704..ff21fc1b457 100644 --- a/docs/es/query_language/misc.md +++ b/docs/es/query_language/misc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Consultas Misceláneas {#miscellaneous-queries} ## CONECTAR {#attach} diff --git a/docs/es/query_language/operators.md b/docs/es/query_language/operators.md index 248e40a62f9..19d6d058f00 100644 --- a/docs/es/query_language/operators.md +++ b/docs/es/query_language/operators.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Operador {#operators} Todos los operadores se transforman a sus funciones correspondientes en la etapa de análisis de consulta de acuerdo con su precedencia y asociatividad. diff --git a/docs/es/query_language/select.md b/docs/es/query_language/select.md index e886b0f0fcb..b3aaef02b79 100644 --- a/docs/es/query_language/select.md +++ b/docs/es/query_language/select.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # SELECCIONAR consultas Sintaxis {#select-queries-syntax} `SELECT` realiza la recuperación de datos. diff --git a/docs/es/query_language/show.md b/docs/es/query_language/show.md index 0341f654035..f713fbe337e 100644 --- a/docs/es/query_language/show.md +++ b/docs/es/query_language/show.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # MOSTRAR consultas {#show-queries} ## MOSTRAR CREAR TABLA {#show-create-table} diff --git a/docs/es/query_language/syntax.md b/docs/es/query_language/syntax.md index b0467b44f77..cb24cf46430 100644 --- a/docs/es/query_language/syntax.md +++ b/docs/es/query_language/syntax.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Sintaxis {#syntax} Hay dos tipos de analizadores en el sistema: el analizador SQL completo (un analizador de descenso recursivo) y el analizador de formato de datos (un analizador de flujo rápido). diff --git a/docs/es/query_language/system.md b/docs/es/query_language/system.md index 27a3f61f20d..bd30ef013ed 100644 --- a/docs/es/query_language/system.md +++ b/docs/es/query_language/system.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Consultas del sistema {#query-language-system} - [Cargar DICCIONARIOS](#query_language-system-reload-dictionaries) diff --git a/docs/es/query_language/table_functions/file.md b/docs/es/query_language/table_functions/file.md index c3744b62e74..f20a208d920 100644 --- a/docs/es/query_language/table_functions/file.md +++ b/docs/es/query_language/table_functions/file.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # file {#file} Crea una tabla a partir de un archivo. Esta función de tabla es similar a [URL](url.md) y [Hdfs](hdfs.md) Aquel. diff --git a/docs/es/query_language/table_functions/generate.md b/docs/es/query_language/table_functions/generate.md index cfaf38ccf9e..5192b0563f3 100644 --- a/docs/es/query_language/table_functions/generate.md +++ b/docs/es/query_language/table_functions/generate.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # generateRandom {#generaterandom} Genera datos aleatorios con un esquema dado. diff --git a/docs/es/query_language/table_functions/hdfs.md b/docs/es/query_language/table_functions/hdfs.md index 9a72d684039..81e9a9e8898 100644 --- a/docs/es/query_language/table_functions/hdfs.md +++ b/docs/es/query_language/table_functions/hdfs.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Hdfs {#hdfs} Crea una tabla a partir de archivos en HDFS. Esta función de tabla es similar a [URL](url.md) y [file](file.md) Aquel. diff --git a/docs/es/query_language/table_functions/index.md b/docs/es/query_language/table_functions/index.md index 7b297676d38..0f249638a0b 100644 --- a/docs/es/query_language/table_functions/index.md +++ b/docs/es/query_language/table_functions/index.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Funciones de tabla {#table-functions} Las funciones de tabla son métodos para construir tablas. diff --git a/docs/es/query_language/table_functions/input.md b/docs/es/query_language/table_functions/input.md index ab89c198c42..cbc91842eb2 100644 --- a/docs/es/query_language/table_functions/input.md +++ b/docs/es/query_language/table_functions/input.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # entrada {#input} `input(structure)` - función de tabla que permite convertir e insertar efectivamente los datos enviados al diff --git a/docs/es/query_language/table_functions/jdbc.md b/docs/es/query_language/table_functions/jdbc.md index ed13151a713..5c97bb56a42 100644 --- a/docs/es/query_language/table_functions/jdbc.md +++ b/docs/es/query_language/table_functions/jdbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # jdbc {#table-function-jdbc} `jdbc(jdbc_connection_uri, schema, table)` - devuelve la tabla que está conectado a través del controlador JDBC. diff --git a/docs/es/query_language/table_functions/merge.md b/docs/es/query_language/table_functions/merge.md index f1090deca81..268ad0a84e0 100644 --- a/docs/es/query_language/table_functions/merge.md +++ b/docs/es/query_language/table_functions/merge.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Fusionar {#merge} `merge(db_name, 'tables_regexp')` – Crea una tabla de combinación temporal. Para obtener más información, consulte la sección “Table engines, Merge”. diff --git a/docs/es/query_language/table_functions/mysql.md b/docs/es/query_language/table_functions/mysql.md index 9fa4d272b63..a8d57279d6b 100644 --- a/docs/es/query_language/table_functions/mysql.md +++ b/docs/es/query_language/table_functions/mysql.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # mysql {#mysql} Permitir `SELECT` consultas que se realizarán en los datos que se almacenan en un servidor MySQL remoto. diff --git a/docs/es/query_language/table_functions/numbers.md b/docs/es/query_language/table_functions/numbers.md index 59d67cc6ff8..5d6017e523c 100644 --- a/docs/es/query_language/table_functions/numbers.md +++ b/docs/es/query_language/table_functions/numbers.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # número {#numbers} `numbers(N)` – Devuelve una tabla con el único ‘number’ columna (UInt64) que contiene enteros de 0 a N-1. diff --git a/docs/es/query_language/table_functions/odbc.md b/docs/es/query_language/table_functions/odbc.md index 13a77aa90d5..1e0cf1a723e 100644 --- a/docs/es/query_language/table_functions/odbc.md +++ b/docs/es/query_language/table_functions/odbc.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Nosotros {#table-functions-odbc} Devuelve la tabla que está conectada a través de [ODBC](https://en.wikipedia.org/wiki/Open_Database_Connectivity). diff --git a/docs/es/query_language/table_functions/remote.md b/docs/es/query_language/table_functions/remote.md index 922c038b1e2..adc7aa44207 100644 --- a/docs/es/query_language/table_functions/remote.md +++ b/docs/es/query_language/table_functions/remote.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Remoto, remoteSecure {#remote-remotesecure} Le permite acceder a servidores remotos sin crear un `Distributed` tabla. diff --git a/docs/es/query_language/table_functions/url.md b/docs/es/query_language/table_functions/url.md index fb7cee98f9a..6978d5ece18 100644 --- a/docs/es/query_language/table_functions/url.md +++ b/docs/es/query_language/table_functions/url.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # URL {#url} `url(URL, format, structure)` - devuelve una tabla creada a partir del `URL` con dado diff --git a/docs/es/roadmap.md b/docs/es/roadmap.md index 6bd63d64a8c..11934c67422 100644 --- a/docs/es/roadmap.md +++ b/docs/es/roadmap.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + # Hoja de ruta {#roadmap} ## Q1 2020 {#q1-2020} diff --git a/docs/es/security_changelog.md b/docs/es/security_changelog.md index 1658c348279..94a0fd83f73 100644 --- a/docs/es/security_changelog.md +++ b/docs/es/security_changelog.md @@ -1,3 +1,7 @@ +--- +machine_translated: true +--- + ## Corregido en la versión de ClickHouse 19.14.3.3, 2019-09-10 {#fixed-in-clickhouse-release-19-14-3-3-2019-09-10} ### ¿Qué puedes encontrar en Neodigit {#cve-2019-15024} diff --git a/docs/fa/getting_started/playground.md b/docs/fa/getting_started/playground.md new file mode 120000 index 00000000000..de5b41f453e --- /dev/null +++ b/docs/fa/getting_started/playground.md @@ -0,0 +1 @@ +../../en/getting_started/playground.md \ No newline at end of file diff --git a/docs/ja/getting_started/playground.md b/docs/ja/getting_started/playground.md new file mode 120000 index 00000000000..de5b41f453e --- /dev/null +++ b/docs/ja/getting_started/playground.md @@ -0,0 +1 @@ +../../en/getting_started/playground.md \ No newline at end of file diff --git a/docs/tools/build.py b/docs/tools/build.py index de496a25cef..10facd5dd10 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -1,5 +1,5 @@ #!/usr/bin/env python3 -# -*- coding: utf-8 -*- + from __future__ import unicode_literals import argparse @@ -11,6 +11,7 @@ import subprocess import sys import time +import jinja2 import livereload import markdown.util @@ -53,9 +54,10 @@ def build_for_lang(lang, args): try: theme_cfg = { 'name': None, - 'custom_dir': os.path.join(os.path.dirname(__file__), args.theme_dir), + 'custom_dir': os.path.join(os.path.dirname(__file__), '..', args.theme_dir), 'language': lang, 'direction': 'rtl' if lang == 'fa' else 'ltr', + # TODO: cleanup 'feature': { 'tabs': False }, @@ -66,8 +68,6 @@ def build_for_lang(lang, args): 'font': False, 'logo': 'images/logo.svg', 'favicon': 'assets/images/favicon.ico', - 'include_search_page': False, - 'search_index_only': False, 'static_templates': ['404.html'], 'extra': { 'now': int(time.mktime(datetime.datetime.now().timetuple())) # TODO better way to avoid caching @@ -83,21 +83,48 @@ def build_for_lang(lang, args): 'fa': 'مستندات %sClickHouse' } + languages = { + 'en': 'English', + 'zh': '中文', + 'es': 'Español', + 'ru': 'Русский', + 'ja': '日本語', + 'fa': 'فارسی' + } + + assert len(site_names) == len(languages) + if args.version_prefix: site_dir = os.path.join(args.docs_output_dir, args.version_prefix, lang) else: site_dir = os.path.join(args.docs_output_dir, lang) - plugins = ['search'] - if not args.no_docs_macros: - plugins.append('macros') + markdown_extensions = [ + 'mdx_clickhouse', + 'admonition', + 'attr_list', + 'codehilite', + 'nl2br', + 'sane_lists', + 'pymdownx.magiclink', + 'pymdownx.superfences', + 'extra', + { + 'toc': { + 'permalink': True, + 'slugify': mdx_clickhouse.slugify + } + } + ] + + plugins = ['macros'] if args.htmlproofer: plugins.append('htmlproofer') - cfg = config.load_config( + raw_config = dict( config_file=config_path, site_name=site_names.get(lang, site_names['en']) % args.version_prefix, - site_url=f'https://clickhouse.yandex/docs/{lang}/', + site_url=f'https://clickhouse.tech/docs/{lang}/', docs_dir=os.path.join(args.docs_dir, lang), site_dir=site_dir, strict=not args.version_prefix, @@ -108,39 +135,35 @@ def build_for_lang(lang, args): repo_url='https://github.com/ClickHouse/ClickHouse/', edit_uri=f'edit/master/docs/{lang}', extra_css=[f'assets/stylesheets/custom.css?{args.rev_short}'], - markdown_extensions=[ - 'mdx_clickhouse', - 'admonition', - 'attr_list', - 'codehilite', - 'nl2br', - 'sane_lists', - 'pymdownx.magiclink', - 'pymdownx.superfences', - 'extra', - { - 'toc': { - 'permalink': True, - 'slugify': mdx_clickhouse.slugify - } - } - ], + markdown_extensions=markdown_extensions, plugins=plugins, extra={ 'stable_releases': args.stable_releases, 'version_prefix': args.version_prefix, + 'single_page': False, 'rev': args.rev, 'rev_short': args.rev_short, 'rev_url': args.rev_url, - 'events': args.events + 'events': args.events, + 'languages': languages } ) - mkdocs_build.build(cfg) + cfg = config.load_config(**raw_config) + + try: + mkdocs_build.build(cfg) + except jinja2.exceptions.TemplateError: + if not args.version_prefix: + raise + mdx_clickhouse.PatchedMacrosPlugin.disabled = True + mkdocs_build.build(cfg) if not args.skip_single_page: build_single_page_version(lang, args, cfg) - + + mdx_clickhouse.PatchedMacrosPlugin.disabled = False + logging.info(f'Finished building {lang} docs') except exceptions.ConfigurationError as e: @@ -150,6 +173,8 @@ def build_for_lang(lang, args): def build_single_page_version(lang, args, cfg): logging.info(f'Building single page version for {lang}') os.environ['SINGLE_PAGE'] = '1' + extra = cfg.data['extra'] + extra['single_page'] = True with util.autoremoved_file(os.path.join(args.docs_dir, lang, 'single.md')) as single_md: concatenate(lang, args.docs_dir, single_md) @@ -167,9 +192,7 @@ def build_single_page_version(lang, args, cfg): cfg.load_dict({ 'docs_dir': docs_temp_lang, 'site_dir': site_temp, - 'extra': { - 'single_page': True - }, + 'extra': extra, 'nav': [ {cfg.data.get('site_name'): 'single.md'} ] @@ -199,12 +222,11 @@ def build_single_page_version(lang, args, cfg): if not args.version_prefix: # maybe enable in future with util.temp_dir() as test_dir: + extra['single_page'] = False cfg.load_dict({ 'docs_dir': docs_temp_lang, 'site_dir': test_dir, - 'extra': { - 'single_page': False - }, + 'extra': extra, 'nav': [ {cfg.data.get('site_name'): 'single.md'} ] @@ -218,6 +240,11 @@ def build_single_page_version(lang, args, cfg): def write_redirect_html(out_path, to_url): + out_dir = os.path.dirname(out_path) + try: + os.makedirs(out_dir) + except OSError: + pass with open(out_path, 'w') as f: f.write(f''' @@ -238,11 +265,6 @@ def write_redirect_html(out_path, to_url): def build_redirect_html(args, from_path, to_path): for lang in args.lang.split(','): out_path = os.path.join(args.docs_output_dir, lang, from_path.replace('.md', '/index.html')) - out_dir = os.path.dirname(out_path) - try: - os.makedirs(out_dir) - except OSError: - pass version_prefix = args.version_prefix + '/' if args.version_prefix else '/' target_path = to_path.replace('.md', '/') to_url = f'/docs{version_prefix}{lang}/{target_path}' @@ -251,19 +273,10 @@ def build_redirect_html(args, from_path, to_path): def build_redirects(args): - lang_re_fragment = args.lang.replace(',', '|') - rewrites = [] - with open(os.path.join(args.docs_dir, 'redirects.txt'), 'r') as f: for line in f: from_path, to_path = line.split(' ', 1) build_redirect_html(args, from_path, to_path) - from_path = '^/docs/(' + lang_re_fragment + ')/' + from_path.replace('.md', '/?') + '$' - to_path = '/docs/$1/' + to_path.replace('.md', '/') - rewrites.append(' '.join(['rewrite', from_path, to_path, 'permanent;'])) - - with open(os.path.join(args.docs_output_dir, 'redirects.conf'), 'w') as f: - f.write('\n'.join(rewrites)) def build_docs(args): @@ -304,14 +317,15 @@ def build(args): if __name__ == '__main__': os.chdir(os.path.join(os.path.dirname(__file__), '..')) - + website_dir = os.path.join('..', 'website') arg_parser = argparse.ArgumentParser() arg_parser.add_argument('--lang', default='en,es,ru,zh,ja,fa') arg_parser.add_argument('--docs-dir', default='.') - arg_parser.add_argument('--theme-dir', default='mkdocs-material-theme') - arg_parser.add_argument('--website-dir', default=os.path.join('..', 'website')) + arg_parser.add_argument('--theme-dir', default=website_dir) + arg_parser.add_argument('--website-dir', default=website_dir) arg_parser.add_argument('--output-dir', default='build') arg_parser.add_argument('--enable-stable-releases', action='store_true') + arg_parser.add_argument('--stable-releases-limit', type=int, default='10') arg_parser.add_argument('--version-prefix', type=str, default='') arg_parser.add_argument('--is-stable-release', action='store_true') arg_parser.add_argument('--skip-single-page', action='store_true') @@ -336,7 +350,7 @@ if __name__ == '__main__': args.docs_output_dir = os.path.join(os.path.abspath(args.output_dir), 'docs') from github import choose_latest_releases, get_events - args.stable_releases = choose_latest_releases() if args.enable_stable_releases else [] + args.stable_releases = choose_latest_releases(args) if args.enable_stable_releases else [] args.rev = subprocess.check_output('git rev-parse HEAD', shell=True).decode('utf-8').strip() args.rev_short = subprocess.check_output('git rev-parse --short HEAD', shell=True).decode('utf-8').strip() args.rev_url = f'https://github.com/ClickHouse/ClickHouse/commit/{args.rev}' @@ -350,10 +364,11 @@ if __name__ == '__main__': new_args = sys.executable + ' ' + ' '.join(new_args) server = livereload.Server() - server.watch(args.website_dir + '**/*', livereload.shell(new_args, cwd='tools', shell=True)) server.watch(args.docs_dir + '**/*', livereload.shell(new_args, cwd='tools', shell=True)) + server.watch(args.website_dir + '**/*', livereload.shell(new_args, cwd='tools', shell=True)) server.serve( root=args.output_dir, + host='0.0.0.0', port=args.livereload ) sys.exit(0) diff --git a/docs/tools/concatenate.py b/docs/tools/concatenate.py index 8b424a0b672..cfbc2acbb4c 100755 --- a/docs/tools/concatenate.py +++ b/docs/tools/concatenate.py @@ -42,14 +42,18 @@ def concatenate(lang, docs_path, single_page_file): for anchor in anchors: if re.search(az_re, anchor): - single_page_file.write('\n' % anchor) + single_page_file.write('' % anchor) - single_page_file.write('\n\n') + single_page_file.write('\n') + in_metadata = False for l in f: + if l.startswith('---'): + in_metadata = not in_metadata if l.startswith('#'): l = '#' + l - single_page_file.write(l) + if not in_metadata: + single_page_file.write(l) except IOError as e: logging.warning(str(e)) diff --git a/docs/tools/github.py b/docs/tools/github.py index 4a02981b4b3..d32c78abb14 100644 --- a/docs/tools/github.py +++ b/docs/tools/github.py @@ -6,17 +6,16 @@ import os import sys import tarfile -import jinja2 import requests import util -def choose_latest_releases(): +def choose_latest_releases(args): logging.info('Collecting release candidates') seen = collections.OrderedDict() candidates = [] - for page in range(1, 10): + for page in range(1, args.stable_releases_limit): url = 'https://api.github.com/repos/ClickHouse/ClickHouse/tags?per_page=100&page=%d' % page candidates += requests.get(url).json() logging.info('Collected all release candidates') @@ -31,7 +30,7 @@ def choose_latest_releases(): major_version = '.'.join((name.split('.', 2))[:2]) if major_version not in seen: seen[major_version] = (name, tag.get('tarball_url'),) - if len(seen) > 10: + if len(seen) > args.stable_releases_limit: break else: logging.fatal('Unexpected GitHub response: %s', str(candidates)) @@ -52,11 +51,7 @@ def process_release(args, callback, release): args.version_prefix = name args.is_stable_release = True args.docs_dir = os.path.join(base_dir, os.listdir(base_dir)[0], 'docs') - try: - callback(args) - except jinja2.exceptions.TemplateSyntaxError: - args.no_docs_macros = True - callback(args) + callback(args) def build_releases(args, callback): diff --git a/docs/tools/make_links.sh b/docs/tools/make_links.sh index 87b3f56a166..dfe9c56615d 100755 --- a/docs/tools/make_links.sh +++ b/docs/tools/make_links.sh @@ -6,7 +6,7 @@ function do_make_links() { - langs=(en es ru zh ja fa) + langs=(en es zh fr ru ja fa) src_file="$1" for lang in "${langs[@]}" do diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index bfe5ebbd1ae..f024c129462 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -11,7 +11,6 @@ import macros.plugin import slugify as slugify_impl - class ClickHouseLinkMixin(object): def handleMatch(self, m, data): @@ -26,7 +25,7 @@ class ClickHouseLinkMixin(object): is_external = href.startswith('http:') or href.startswith('https:') if is_external: if not href.startswith('https://clickhouse.tech'): - el.set('rel', 'external nofollow') + el.set('rel', 'external nofollow noreferrer') elif single_page: if '#' in href: el.set('href', '#' + href.split('#', 1)[1]) @@ -70,11 +69,43 @@ def slugify(value, separator): return slugify_impl.slugify(value, separator=separator, word_boundary=True, save_order=True) +def get_translations(dirname, lang): + import babel.support + return babel.support.Translations.load( + dirname=dirname, + locales=[lang, 'en'] + ) + + class PatchedMacrosPlugin(macros.plugin.MacrosPlugin): + disabled = False + def on_config(self, config): super(PatchedMacrosPlugin, self).on_config(config) self.env.comment_start_string = '{##' self.env.comment_end_string = '##}' + def on_env(self, env, config, files): + env.add_extension('jinja2.ext.i18n') + dirname = os.path.join(config.data['theme'].dirs[0], 'locale') + lang = config.data['theme']['language'] + env.install_gettext_translations( + get_translations(dirname, lang), + newstyle=True + ) + chunk_size = 10240 + env.filters['chunks'] = lambda line: [line[i:i+chunk_size] for i in range(0, len(line), chunk_size)] + return env + + def render(self, markdown): + if not self.disabled: + return self.render_impl(markdown) + else: + return markdown + + def render_impl(self, markdown): + md_template = self.env.from_string(markdown) + return md_template.render(**self.variables) + macros.plugin.MacrosPlugin = PatchedMacrosPlugin diff --git a/docs/tools/mkdocs-material-theme/partials/nav.html b/docs/tools/mkdocs-material-theme/partials/nav.html index cdc09492e54..9bf1076c349 100644 --- a/docs/tools/mkdocs-material-theme/partials/nav.html +++ b/docs/tools/mkdocs-material-theme/partials/nav.html @@ -41,7 +41,7 @@ {% if config.repo_url %}
  • - + {{ lang.t("nav.source") }}
  • diff --git a/docs/tools/mkdocs-material-theme/partials/social.html b/docs/tools/mkdocs-material-theme/partials/social.html index 89e6f0fec87..b990921bb8a 100644 --- a/docs/tools/mkdocs-material-theme/partials/social.html +++ b/docs/tools/mkdocs-material-theme/partials/social.html @@ -1,3 +1,3 @@ diff --git a/docs/tools/release.sh b/docs/tools/release.sh index bf318f17252..e0f580c383b 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -7,8 +7,8 @@ PUBLISH_DIR="${BASE_DIR}/../publish" BASE_DOMAIN="${BASE_DOMAIN:-clickhouse.tech}" GIT_TEST_URI="${GIT_TEST_URI:-git@github.com:ClickHouse/clickhouse.github.io.git}" GIT_PROD_URI="git@github.com:ClickHouse/clickhouse.github.io.git" -EXTRA_BUILD_ARGS="${EXTRA_BUILD_ARGS:---enable-stable-releases}" -HISTORY_SIZE="${HISTORY_SIZE:-25}" +EXTRA_BUILD_ARGS="${EXTRA_BUILD_ARGS:---enable-stable-releases --minify}" +HISTORY_SIZE="${HISTORY_SIZE:-5}" if [[ -z "$1" ]] then diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index a86e15cec19..33cf57d41bb 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -1,9 +1,11 @@ +Babel==2.8.0 backports-abc==0.5 backports.functools-lru-cache==1.6.1 beautifulsoup4==4.8.2 certifi==2019.11.28 chardet==3.0.4 click==7.1.1 +closure==20191111 cssmin==0.2.0 future==0.18.2 htmlmin==0.1.12 diff --git a/docs/tools/test.py b/docs/tools/test.py index bb2e19e8760..58710274543 100755 --- a/docs/tools/test.py +++ b/docs/tools/test.py @@ -26,7 +26,7 @@ def test_single_page(input_path, lang): anchor_points.add(anchor_point) for tag in soup.find_all(): href = tag.attrs.get('href') - if href and href.startswith('#'): + if href and href.startswith('#') and href != '#': if href[1:] not in anchor_points: links_to_nowhere += 1 logging.info("Tag %s", tag) diff --git a/docs/tools/translate/babel-mapping.ini b/docs/tools/translate/babel-mapping.ini new file mode 100644 index 00000000000..6a9a3e5c073 --- /dev/null +++ b/docs/tools/translate/babel-mapping.ini @@ -0,0 +1,3 @@ +[python: **.py] +[jinja2: **/templates/**.html] +extensions=jinja2.ext.i18n,jinja2.ext.autoescape,jinja2.ext.with_ diff --git a/docs/tools/translate/filter.py b/docs/tools/translate/filter.py index cf673a75d63..c0cb04f059f 100755 --- a/docs/tools/translate/filter.py +++ b/docs/tools/translate/filter.py @@ -18,7 +18,7 @@ def debug(*args): print(*args, file=sys.stderr) -def process_buffer(buffer, new_value, item=None): +def process_buffer(buffer, new_value, item=None, is_header=False): if buffer: text = ''.join(buffer) @@ -37,11 +37,17 @@ def process_buffer(buffer, new_value, item=None): if text.startswith(' ') and not translated_text.startswith(' '): translated_text = ' ' + translated_text - + if text.endswith(' ') and not translated_text.endswith(' '): translated_text = translated_text + ' ' + title_case = is_header and translate.default_target_language == 'en' and text[0].isupper() + title_case_whitelist = {'a', 'an', 'the', 'and', 'or'} for token in translated_text.split(' '): + if title_case and not token.isupper(): + if token not in title_case_whitelist: + token = token[0].upper() + token[1:] + new_value.append(pandocfilters.Str(token)) new_value.append(pandocfilters.Space()) @@ -53,12 +59,12 @@ def process_buffer(buffer, new_value, item=None): new_value.append(item) -def process_sentence(value): +def process_sentence(value, is_header=False): new_value = [] buffer = [] for item in value: if isinstance(item, list): - new_value.append([process_sentence(subitem) for subitem in item]) + new_value.append([process_sentence(subitem, is_header) for subitem in item]) continue elif isinstance(item, dict): t = item.get('t') @@ -70,11 +76,11 @@ def process_sentence(value): elif t == 'DoubleQuote': buffer.append('"') else: - process_buffer(buffer, new_value, item) + process_buffer(buffer, new_value, item, is_header) buffer = [] else: new_value.append(item) - process_buffer(buffer, new_value) + process_buffer(buffer, new_value, is_header=is_header) return new_value @@ -136,10 +142,11 @@ def translate_filter(key, value, _format, _): value[1] = process_sentence(value[1]) return cls(*value) elif key == 'Header': - # TODO: title case header in en - if '_' not in value[1][0]: # Preserve some manually specified anchors + if value[1][0].islower() and '_' not in value[1][0]: # Preserve some manually specified anchors value[1][0] = slugify.slugify(value[1][0], separator='-', word_boundary=True, save_order=True) - value[2] = process_sentence(value[2]) + + # TODO: title case header in en + value[2] = process_sentence(value[2], is_header=True) return cls(*value) elif key == 'SoftBreak': return pandocfilters.LineBreak() diff --git a/docs/tools/translate/requirements.txt b/docs/tools/translate/requirements.txt index 975498ac012..41b1db836d3 100644 --- a/docs/tools/translate/requirements.txt +++ b/docs/tools/translate/requirements.txt @@ -1,7 +1,9 @@ +Babel==2.8.0 certifi==2019.11.28 chardet==3.0.4 googletrans==2.4.0 idna==2.9 +Jinja2==2.11.1 pandocfilters==1.4.2 python-slugify==4.0.0 PyYAML==5.3.1 diff --git a/docs/tools/translate/translate.py b/docs/tools/translate/translate.py index 03445defcc6..c3767e7990b 100755 --- a/docs/tools/translate/translate.py +++ b/docs/tools/translate/translate.py @@ -14,12 +14,13 @@ import typograph_ru translator = googletrans.Translator() -target_language = os.environ.get('TARGET_LANGUAGE', 'ru') +default_target_language = os.environ.get('TARGET_LANGUAGE', 'ru') is_yandex = os.environ.get('YANDEX') is not None -def translate(text): +def translate(text, target_language=None): + target_language = target_language or default_target_language if target_language == 'en': return text elif target_language == 'typograph_ru': @@ -27,14 +28,16 @@ def translate(text): elif is_yandex: text = text.replace('‘', '\'') text = text.replace('’', '\'') - if text.isascii() and not text.isupper(): + has_alpha = any([char.isalpha() for char in text]) + if text.isascii() and has_alpha and not text.isupper(): text = urllib.parse.quote(text) url = f'http://translate.yandex.net/api/v1/tr.json/translate?srv=docs&lang=en-{target_language}&text={text}' result = requests.get(url).json() if result.get('code') == 200: return result['text'][0] else: - print('Failed to translate', str(result), file=sys.stderr) + result = str(result) + print(f'Failed to translate "{text}": {result}', file=sys.stderr) sys.exit(1) else: return text @@ -56,8 +59,27 @@ def translate_toc(root): return root +def translate_po(): + import babel.messages.pofile + base_dir = os.path.join(os.path.dirname(__file__), '..', '..', 'website', 'locale') + for lang in ['en', 'zh', 'es', 'ru', 'ja', 'fa']: + po_path = os.path.join(base_dir, lang, 'LC_MESSAGES', 'messages.po') + with open(po_path, 'r') as f: + po_file = babel.messages.pofile.read_po(f, locale=lang, domain='messages') + for item in po_file: + if not item.string: + global is_yandex + is_yandex = True + item.string = translate(item.id, lang) + with open(po_path, 'wb') as f: + babel.messages.pofile.write_po(f, po_file) + + if __name__ == '__main__': target_language = sys.argv[1] - is_yandex = True - result = translate_toc(yaml.full_load(sys.stdin.read())['nav']) - print(yaml.dump({'nav': result})) + if target_language == 'po': + translate_po() + else: + is_yandex = True + result = translate_toc(yaml.full_load(sys.stdin.read())['nav']) + print(yaml.dump({'nav': result})) diff --git a/docs/tools/translate/update-po.sh b/docs/tools/translate/update-po.sh new file mode 100755 index 00000000000..81c37299323 --- /dev/null +++ b/docs/tools/translate/update-po.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Usage: update-po.sh +set -ex +BASE_DIR=$(dirname $(readlink -f $0)) +WEBSITE_DIR="${BASE_DIR}/../../website" +LOCALE_DIR="${WEBSITE_DIR}/locale" +MESSAGES_POT="${LOCALE_DIR}/messages.pot" +BABEL_INI="${BASE_DIR}/babel-mapping.ini" +LANGS="en ru zh ja fa es" +source "${BASE_DIR}/venv/bin/activate" +cd "${WEBSITE_DIR}" +pybabel extract "." -o "${MESSAGES_POT}" -F "${BABEL_INI}" +for L in ${LANGS} +do + pybabel update -d locale -l "${L}" -i "${MESSAGES_POT}" || \ + pybabel init -d locale -l "${L}" -i "${MESSAGES_POT}" +done +python3 "${BASE_DIR}/translate.py" po +for L in ${LANGS} +do + pybabel compile -d locale -l "${L}" +done diff --git a/docs/tools/website.py b/docs/tools/website.py index 60cc08a60f6..bdcf7720256 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -1,12 +1,16 @@ +import hashlib import logging import os import shutil +import subprocess import cssmin import htmlmin import jinja2 import jsmin +import mdx_clickhouse + def copy_icons(args): logging.info('Copying icons') @@ -27,9 +31,17 @@ def build_website(args): logging.info('Building website') env = jinja2.Environment( loader=jinja2.FileSystemLoader(args.website_dir), - extensions=['jinja2_highlight.HighlightExtension'] + extensions=[ + 'jinja2.ext.i18n', + 'jinja2_highlight.HighlightExtension' + ] ) env.extend(jinja2_highlight_cssclass='syntax p-3 my-3') + translations_dir = os.path.join(args.website_dir, 'locale') + env.install_gettext_translations( + mdx_clickhouse.get_translations(translations_dir, 'en'), + newstyle=True + ) shutil.copytree( args.website_dir, @@ -37,6 +49,8 @@ def build_website(args): ignore=shutil.ignore_patterns( '*.md', '*.sh', + '*.css', + '*.js', 'build', 'docs', 'public', @@ -52,7 +66,7 @@ def build_website(args): continue path = os.path.join(root, filename) - if not (filename.endswith('.html') or filename.endswith('.css') or filename.endswith('.js')): + if not (filename.endswith('.html') or filename.endswith('.js')): continue logging.info('Processing %s', path) with open(path, 'rb') as f: @@ -66,6 +80,56 @@ def build_website(args): def minify_website(args): + css_in = f"'{args.website_dir}/css/bootstrap.css' " \ + f"'{args.website_dir}/css/docsearch.css' " \ + f"'{args.website_dir}/css/base.css' '{args.website_dir}/css/docs.css' " \ + f"'{args.website_dir}/css/highlight.css'" + css_out = f'{args.output_dir}/css/base.css' + if args.minify: + command = f"purifycss -w '*algolia*' --min {css_in} '{args.output_dir}/*.html' " \ + f"'{args.output_dir}/docs/en/**/*.html' '{args.website_dir}/js/**/*.js' > {css_out}" + else: + command = f'cat {css_in} > {css_out}' + logging.info(command) + output = subprocess.check_output(command, shell=True) + logging.debug(output) + with open(css_out, 'rb') as f: + css_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8] + + js_in = [ + f"'{args.website_dir}/js/jquery-3.4.1.js'", + f"'{args.website_dir}/js/popper.js'", + f"'{args.website_dir}/js/bootstrap.js'", + f"'{args.website_dir}/js/base.js'", + f"'{args.website_dir}/js/index.js'", + f"'{args.website_dir}/js/docsearch.js'", + f"'{args.website_dir}/js/docs.js'" + ] + js_out = f'{args.output_dir}/js/base.js' + if args.minify: + import closure + js_in = [js[1:-1] for js in js_in] + closure_args = [ + '--js', *js_in, '--js_output_file', js_out, + '--compilation_level', 'SIMPLE', + '--dependency_mode', 'NONE', + '--third_party', '--use_types_for_optimization', + '--isolation_mode', 'IIFE', + '--create_source_map', '%outname%.map' + ] + logging.info(closure_args) + if closure.run(*closure_args): + raise RuntimeError('failed to run closure compiler') + + else: + logging.info(command) + js_in = ' '.join(js_in) + output = subprocess.check_output(f'cat {js_in} > {js_out}', shell=True) + logging.debug(output) + with open(js_out, 'rb') as f: + js_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8] + logging.info(js_digest) + if args.minify: logging.info('Minifying website') for root, _, filenames in os.walk(args.output_dir): @@ -73,8 +137,7 @@ def minify_website(args): path = os.path.join(root, filename) if not ( filename.endswith('.html') or - filename.endswith('.css') or - filename.endswith('.js') + filename.endswith('.css') ): continue @@ -83,6 +146,8 @@ def minify_website(args): content = f.read().decode('utf-8') if filename.endswith('.html'): content = htmlmin.minify(content, remove_empty_space=False) + content = content.replace('base.css?css_digest', f'base.css?{css_digest}') + content = content.replace('base.js?js_digest', f'base.js?{js_digest}') elif filename.endswith('.css'): content = cssmin.cssmin(content) elif filename.endswith('.js'): diff --git a/docs/zh/getting_started/playground.md b/docs/zh/getting_started/playground.md new file mode 120000 index 00000000000..de5b41f453e --- /dev/null +++ b/docs/zh/getting_started/playground.md @@ -0,0 +1 @@ +../../en/getting_started/playground.md \ No newline at end of file diff --git a/website/404.html b/website/404.html index d7e096e75ff..31aaf6646df 100644 --- a/website/404.html +++ b/website/404.html @@ -4,12 +4,12 @@ {% block content %} {% endblock %} diff --git a/website/css/base.css b/website/css/base.css index e171339f171..4926bd3af8a 100644 --- a/website/css/base.css +++ b/website/css/base.css @@ -11,7 +11,7 @@ a:hover, a:active { text-decoration: underline; } -#logo-icon { +#logo-icon, #docs-logo-icon { width: 40px; } @@ -20,10 +20,40 @@ a:hover, a:active { margin-left: 12px; } +.display-5 { + font-size: 2rem; + font-weight: 300; + line-height: 1.2; +} + +.bg-dark-alt { + background: #36363F; +} + +.bg-dark-alt:focus { + background: #36363F; +} + .bg-secondary-alt { background: #444451; } +.bg-secondary-alt:focus { + background: #444451; +} + +.text-dark-alt { + color: #36363F; +} + +.btn-dark-alt { + background: #36363F; +} + +.btn-dark-alt:focus { + background: #36363F; +} + .btn-outline-orange { border-color: #f14600; color: #f14600; @@ -68,10 +98,11 @@ a.btn-outline-yellow { color: #fc0; } -.navbar { +.navbar-light { border-bottom: 1px solid #efefef; } + .navbar-light .navbar-nav .nav-link { color: #000; } @@ -89,6 +120,28 @@ a.btn-outline-yellow { margin-top: -1rem; } +.dropdown-item { + color: #e1e1e1 !important; +} + +.dropdown-item:hover, +.dropdown-item:focus { + background-color: #1e1e1e; +} + +.nav .dropdown-toggle::after { + vertical-align: 0; +} +.nav .dropdown-toggle[aria-expanded=false]::after { + display: inline-block; + margin-right: .255em; + vertical-align: 0; + content: ""; + border-top: .3em solid transparent; + border-right: .3em solid; + border-bottom: .3em solid transparent; +} + .dots-lb { background: #fff url('/images/dots.svg') no-repeat 0 100%; } @@ -108,5 +161,3 @@ a.btn-outline-yellow { .dots-cc { background: #fff url('/images/dots.svg') repeat-y 50% -17.5%; } - -{% include "css/highlight.css" %} diff --git a/website/css/bootstrap.css b/website/css/bootstrap.css new file mode 100644 index 00000000000..8eac957a512 --- /dev/null +++ b/website/css/bootstrap.css @@ -0,0 +1,10224 @@ +/*! + * Bootstrap v4.4.1 (https://getbootstrap.com/) + * Copyright 2011-2019 The Bootstrap Authors + * Copyright 2011-2019 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ +:root { + --blue: #007bff; + --indigo: #6610f2; + --purple: #6f42c1; + --pink: #e83e8c; + --red: #dc3545; + --orange: #fd7e14; + --yellow: #ffc107; + --green: #28a745; + --teal: #20c997; + --cyan: #17a2b8; + --white: #fff; + --gray: #6c757d; + --gray-dark: #343a40; + --primary: #007bff; + --secondary: #6c757d; + --success: #28a745; + --info: #17a2b8; + --warning: #ffc107; + --danger: #dc3545; + --light: #f8f9fa; + --dark: #343a40; + --breakpoint-xs: 0; + --breakpoint-sm: 576px; + --breakpoint-md: 768px; + --breakpoint-lg: 992px; + --breakpoint-xl: 1200px; + --font-family-sans-serif: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"; + --font-family-monospace: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace; +} + +*, +*::before, +*::after { + box-sizing: border-box; +} + +html { + font-family: sans-serif; + line-height: 1.15; + -webkit-text-size-adjust: 100%; + -webkit-tap-highlight-color: rgba(0, 0, 0, 0); +} + +article, aside, figcaption, figure, footer, header, hgroup, main, nav, section { + display: block; +} + +body { + margin: 0; + font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"; + font-size: 1rem; + font-weight: 400; + line-height: 1.5; + color: #212529; + text-align: left; + background-color: #fff; +} + +[tabindex="-1"]:focus:not(:focus-visible) { + outline: 0 !important; +} + +hr { + box-sizing: content-box; + height: 0; + overflow: visible; +} + +h1, h2, h3, h4, h5, h6 { + margin-top: 0; + margin-bottom: 0.5rem; +} + +p { + margin-top: 0; + margin-bottom: 1rem; +} + +abbr[title], +abbr[data-original-title] { + text-decoration: underline; + -webkit-text-decoration: underline dotted; + text-decoration: underline dotted; + cursor: help; + border-bottom: 0; + -webkit-text-decoration-skip-ink: none; + text-decoration-skip-ink: none; +} + +address { + margin-bottom: 1rem; + font-style: normal; + line-height: inherit; +} + +ol, +ul, +dl { + margin-top: 0; + margin-bottom: 1rem; +} + +ol ol, +ul ul, +ol ul, +ul ol { + margin-bottom: 0; +} + +dt { + font-weight: 700; +} + +dd { + margin-bottom: .5rem; + margin-left: 0; +} + +blockquote { + margin: 0 0 1rem; +} + +b, +strong { + font-weight: bolder; +} + +small { + font-size: 80%; +} + +sub, +sup { + position: relative; + font-size: 75%; + line-height: 0; + vertical-align: baseline; +} + +sub { + bottom: -.25em; +} + +sup { + top: -.5em; +} + +a { + color: #007bff; + text-decoration: none; + background-color: transparent; +} + +a:hover { + color: #0056b3; + text-decoration: underline; +} + +a:not([href]) { + color: inherit; + text-decoration: none; +} + +a:not([href]):hover { + color: inherit; + text-decoration: none; +} + +pre, +code, +kbd, +samp { + font-family: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace; + font-size: 1em; +} + +pre { + margin-top: 0; + margin-bottom: 1rem; + overflow: auto; +} + +figure { + margin: 0 0 1rem; +} + +img { + vertical-align: middle; + border-style: none; +} + +svg { + overflow: hidden; + vertical-align: middle; +} + +table { + border-collapse: collapse; +} + +caption { + padding-top: 0.75rem; + padding-bottom: 0.75rem; + color: #6c757d; + text-align: left; + caption-side: bottom; +} + +th { + text-align: inherit; +} + +label { + display: inline-block; + margin-bottom: 0.5rem; +} + +button { + border-radius: 0; +} + +button:focus { + outline: 1px dotted; + outline: 5px auto -webkit-focus-ring-color; +} + +input, +button, +select, +optgroup, +textarea { + margin: 0; + font-family: inherit; + font-size: inherit; + line-height: inherit; +} + +button, +input { + overflow: visible; +} + +button, +select { + text-transform: none; +} + +select { + word-wrap: normal; +} + +button, +[type="button"], +[type="reset"], +[type="submit"] { + -webkit-appearance: button; +} + +button:not(:disabled), +[type="button"]:not(:disabled), +[type="reset"]:not(:disabled), +[type="submit"]:not(:disabled) { + cursor: pointer; +} + +button::-moz-focus-inner, +[type="button"]::-moz-focus-inner, +[type="reset"]::-moz-focus-inner, +[type="submit"]::-moz-focus-inner { + padding: 0; + border-style: none; +} + +input[type="radio"], +input[type="checkbox"] { + box-sizing: border-box; + padding: 0; +} + +input[type="date"], +input[type="time"], +input[type="datetime-local"], +input[type="month"] { + -webkit-appearance: listbox; +} + +textarea { + overflow: auto; + resize: vertical; +} + +fieldset { + min-width: 0; + padding: 0; + margin: 0; + border: 0; +} + +legend { + display: block; + width: 100%; + max-width: 100%; + padding: 0; + margin-bottom: .5rem; + font-size: 1.5rem; + line-height: inherit; + color: inherit; + white-space: normal; +} + +progress { + vertical-align: baseline; +} + +[type="number"]::-webkit-inner-spin-button, +[type="number"]::-webkit-outer-spin-button { + height: auto; +} + +[type="search"] { + outline-offset: -2px; + -webkit-appearance: none; +} + +[type="search"]::-webkit-search-decoration { + -webkit-appearance: none; +} + +::-webkit-file-upload-button { + font: inherit; + -webkit-appearance: button; +} + +output { + display: inline-block; +} + +summary { + display: list-item; + cursor: pointer; +} + +template { + display: none; +} + +[hidden] { + display: none !important; +} + +h1, h2, h3, h4, h5, h6, +.h1, .h2, .h3, .h4, .h5, .h6 { + margin-bottom: 0.5rem; + font-weight: 500; + line-height: 1.2; +} + +h1, .h1 { + font-size: 2.5rem; +} + +h2, .h2 { + font-size: 2rem; +} + +h3, .h3 { + font-size: 1.75rem; +} + +h4, .h4 { + font-size: 1.5rem; +} + +h5, .h5 { + font-size: 1.25rem; +} + +h6, .h6 { + font-size: 1rem; +} + +.lead { + font-size: 1.25rem; + font-weight: 300; +} + +.display-1 { + font-size: 6rem; + font-weight: 300; + line-height: 1.2; +} + +.display-2 { + font-size: 5.5rem; + font-weight: 300; + line-height: 1.2; +} + +.display-3 { + font-size: 4.5rem; + font-weight: 300; + line-height: 1.2; +} + +.display-4 { + font-size: 3.5rem; + font-weight: 300; + line-height: 1.2; +} + +hr { + margin-top: 1rem; + margin-bottom: 1rem; + border: 0; + border-top: 1px solid rgba(0, 0, 0, 0.1); +} + +small, +.small { + font-size: 80%; + font-weight: 400; +} + +mark, +.mark { + padding: 0.2em; + background-color: #fcf8e3; +} + +.list-unstyled { + padding-left: 0; + list-style: none; +} + +.list-inline { + padding-left: 0; + list-style: none; +} + +.list-inline-item { + display: inline-block; +} + +.list-inline-item:not(:last-child) { + margin-right: 0.5rem; +} + +.initialism { + font-size: 90%; + text-transform: uppercase; +} + +.blockquote { + margin-bottom: 1rem; + font-size: 1.25rem; +} + +.blockquote-footer { + display: block; + font-size: 80%; + color: #6c757d; +} + +.blockquote-footer::before { + content: "\2014\00A0"; +} + +.img-fluid { + max-width: 100%; + height: auto; +} + +.img-thumbnail { + padding: 0.25rem; + background-color: #fff; + border: 1px solid #dee2e6; + border-radius: 0.25rem; + max-width: 100%; + height: auto; +} + +.figure { + display: inline-block; +} + +.figure-img { + margin-bottom: 0.5rem; + line-height: 1; +} + +.figure-caption { + font-size: 90%; + color: #6c757d; +} + +code { + font-size: 87.5%; + color: #e83e8c; + word-wrap: break-word; +} + +a > code { + color: inherit; +} + +kbd { + padding: 0.2rem 0.4rem; + font-size: 87.5%; + color: #fff; + background-color: #212529; + border-radius: 0.2rem; +} + +kbd kbd { + padding: 0; + font-size: 100%; + font-weight: 700; +} + +pre { + display: block; + font-size: 87.5%; + color: #212529; +} + +pre code { + font-size: inherit; + color: inherit; + word-break: normal; +} + +.pre-scrollable { + max-height: 340px; + overflow-y: scroll; +} + +.container { + width: 100%; + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} + +@media (min-width: 576px) { + .container { + max-width: 540px; + } +} + +@media (min-width: 768px) { + .container { + max-width: 720px; + } +} + +@media (min-width: 992px) { + .container { + max-width: 960px; + } +} + +@media (min-width: 1200px) { + .container { + max-width: 1140px; + } +} + +.container-fluid, .container-sm, .container-md, .container-lg, .container-xl { + width: 100%; + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} + +@media (min-width: 576px) { + .container, .container-sm { + max-width: 540px; + } +} + +@media (min-width: 768px) { + .container, .container-sm, .container-md { + max-width: 720px; + } +} + +@media (min-width: 992px) { + .container, .container-sm, .container-md, .container-lg { + max-width: 960px; + } +} + +@media (min-width: 1200px) { + .container, .container-sm, .container-md, .container-lg, .container-xl { + max-width: 1140px; + } +} + +.row { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + margin-right: -15px; + margin-left: -15px; +} + +.no-gutters { + margin-right: 0; + margin-left: 0; +} + +.no-gutters > .col, +.no-gutters > [class*="col-"] { + padding-right: 0; + padding-left: 0; +} + +.col-1, .col-2, .col-3, .col-4, .col-5, .col-6, .col-7, .col-8, .col-9, .col-10, .col-11, .col-12, .col, +.col-auto, .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12, .col-sm, +.col-sm-auto, .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12, .col-md, +.col-md-auto, .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12, .col-lg, +.col-lg-auto, .col-xl-1, .col-xl-2, .col-xl-3, .col-xl-4, .col-xl-5, .col-xl-6, .col-xl-7, .col-xl-8, .col-xl-9, .col-xl-10, .col-xl-11, .col-xl-12, .col-xl, +.col-xl-auto { + position: relative; + width: 100%; + padding-right: 15px; + padding-left: 15px; +} + +.col { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + max-width: 100%; +} + +.row-cols-1 > * { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; +} + +.row-cols-2 > * { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; +} + +.row-cols-3 > * { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; +} + +.row-cols-4 > * { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; +} + +.row-cols-5 > * { + -ms-flex: 0 0 20%; + flex: 0 0 20%; + max-width: 20%; +} + +.row-cols-6 > * { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; +} + +.col-auto { + -ms-flex: 0 0 auto; + flex: 0 0 auto; + width: auto; + max-width: 100%; +} + +.col-1 { + -ms-flex: 0 0 8.333333%; + flex: 0 0 8.333333%; + max-width: 8.333333%; +} + +.col-2 { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; +} + +.col-3 { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; +} + +.col-4 { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; +} + +.col-5 { + -ms-flex: 0 0 41.666667%; + flex: 0 0 41.666667%; + max-width: 41.666667%; +} + +.col-6 { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; +} + +.col-7 { + -ms-flex: 0 0 58.333333%; + flex: 0 0 58.333333%; + max-width: 58.333333%; +} + +.col-8 { + -ms-flex: 0 0 66.666667%; + flex: 0 0 66.666667%; + max-width: 66.666667%; +} + +.col-9 { + -ms-flex: 0 0 75%; + flex: 0 0 75%; + max-width: 75%; +} + +.col-10 { + -ms-flex: 0 0 83.333333%; + flex: 0 0 83.333333%; + max-width: 83.333333%; +} + +.col-11 { + -ms-flex: 0 0 91.666667%; + flex: 0 0 91.666667%; + max-width: 91.666667%; +} + +.col-12 { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; +} + +.order-first { + -ms-flex-order: -1; + order: -1; +} + +.order-last { + -ms-flex-order: 13; + order: 13; +} + +.order-0 { + -ms-flex-order: 0; + order: 0; +} + +.order-1 { + -ms-flex-order: 1; + order: 1; +} + +.order-2 { + -ms-flex-order: 2; + order: 2; +} + +.order-3 { + -ms-flex-order: 3; + order: 3; +} + +.order-4 { + -ms-flex-order: 4; + order: 4; +} + +.order-5 { + -ms-flex-order: 5; + order: 5; +} + +.order-6 { + -ms-flex-order: 6; + order: 6; +} + +.order-7 { + -ms-flex-order: 7; + order: 7; +} + +.order-8 { + -ms-flex-order: 8; + order: 8; +} + +.order-9 { + -ms-flex-order: 9; + order: 9; +} + +.order-10 { + -ms-flex-order: 10; + order: 10; +} + +.order-11 { + -ms-flex-order: 11; + order: 11; +} + +.order-12 { + -ms-flex-order: 12; + order: 12; +} + +.offset-1 { + margin-left: 8.333333%; +} + +.offset-2 { + margin-left: 16.666667%; +} + +.offset-3 { + margin-left: 25%; +} + +.offset-4 { + margin-left: 33.333333%; +} + +.offset-5 { + margin-left: 41.666667%; +} + +.offset-6 { + margin-left: 50%; +} + +.offset-7 { + margin-left: 58.333333%; +} + +.offset-8 { + margin-left: 66.666667%; +} + +.offset-9 { + margin-left: 75%; +} + +.offset-10 { + margin-left: 83.333333%; +} + +.offset-11 { + margin-left: 91.666667%; +} + +@media (min-width: 576px) { + .col-sm { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + max-width: 100%; + } + .row-cols-sm-1 > * { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .row-cols-sm-2 > * { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .row-cols-sm-3 > * { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .row-cols-sm-4 > * { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .row-cols-sm-5 > * { + -ms-flex: 0 0 20%; + flex: 0 0 20%; + max-width: 20%; + } + .row-cols-sm-6 > * { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-sm-auto { + -ms-flex: 0 0 auto; + flex: 0 0 auto; + width: auto; + max-width: 100%; + } + .col-sm-1 { + -ms-flex: 0 0 8.333333%; + flex: 0 0 8.333333%; + max-width: 8.333333%; + } + .col-sm-2 { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-sm-3 { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .col-sm-4 { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .col-sm-5 { + -ms-flex: 0 0 41.666667%; + flex: 0 0 41.666667%; + max-width: 41.666667%; + } + .col-sm-6 { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .col-sm-7 { + -ms-flex: 0 0 58.333333%; + flex: 0 0 58.333333%; + max-width: 58.333333%; + } + .col-sm-8 { + -ms-flex: 0 0 66.666667%; + flex: 0 0 66.666667%; + max-width: 66.666667%; + } + .col-sm-9 { + -ms-flex: 0 0 75%; + flex: 0 0 75%; + max-width: 75%; + } + .col-sm-10 { + -ms-flex: 0 0 83.333333%; + flex: 0 0 83.333333%; + max-width: 83.333333%; + } + .col-sm-11 { + -ms-flex: 0 0 91.666667%; + flex: 0 0 91.666667%; + max-width: 91.666667%; + } + .col-sm-12 { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .order-sm-first { + -ms-flex-order: -1; + order: -1; + } + .order-sm-last { + -ms-flex-order: 13; + order: 13; + } + .order-sm-0 { + -ms-flex-order: 0; + order: 0; + } + .order-sm-1 { + -ms-flex-order: 1; + order: 1; + } + .order-sm-2 { + -ms-flex-order: 2; + order: 2; + } + .order-sm-3 { + -ms-flex-order: 3; + order: 3; + } + .order-sm-4 { + -ms-flex-order: 4; + order: 4; + } + .order-sm-5 { + -ms-flex-order: 5; + order: 5; + } + .order-sm-6 { + -ms-flex-order: 6; + order: 6; + } + .order-sm-7 { + -ms-flex-order: 7; + order: 7; + } + .order-sm-8 { + -ms-flex-order: 8; + order: 8; + } + .order-sm-9 { + -ms-flex-order: 9; + order: 9; + } + .order-sm-10 { + -ms-flex-order: 10; + order: 10; + } + .order-sm-11 { + -ms-flex-order: 11; + order: 11; + } + .order-sm-12 { + -ms-flex-order: 12; + order: 12; + } + .offset-sm-0 { + margin-left: 0; + } + .offset-sm-1 { + margin-left: 8.333333%; + } + .offset-sm-2 { + margin-left: 16.666667%; + } + .offset-sm-3 { + margin-left: 25%; + } + .offset-sm-4 { + margin-left: 33.333333%; + } + .offset-sm-5 { + margin-left: 41.666667%; + } + .offset-sm-6 { + margin-left: 50%; + } + .offset-sm-7 { + margin-left: 58.333333%; + } + .offset-sm-8 { + margin-left: 66.666667%; + } + .offset-sm-9 { + margin-left: 75%; + } + .offset-sm-10 { + margin-left: 83.333333%; + } + .offset-sm-11 { + margin-left: 91.666667%; + } +} + +@media (min-width: 768px) { + .col-md { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + max-width: 100%; + } + .row-cols-md-1 > * { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .row-cols-md-2 > * { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .row-cols-md-3 > * { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .row-cols-md-4 > * { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .row-cols-md-5 > * { + -ms-flex: 0 0 20%; + flex: 0 0 20%; + max-width: 20%; + } + .row-cols-md-6 > * { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-md-auto { + -ms-flex: 0 0 auto; + flex: 0 0 auto; + width: auto; + max-width: 100%; + } + .col-md-1 { + -ms-flex: 0 0 8.333333%; + flex: 0 0 8.333333%; + max-width: 8.333333%; + } + .col-md-2 { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-md-3 { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .col-md-4 { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .col-md-5 { + -ms-flex: 0 0 41.666667%; + flex: 0 0 41.666667%; + max-width: 41.666667%; + } + .col-md-6 { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .col-md-7 { + -ms-flex: 0 0 58.333333%; + flex: 0 0 58.333333%; + max-width: 58.333333%; + } + .col-md-8 { + -ms-flex: 0 0 66.666667%; + flex: 0 0 66.666667%; + max-width: 66.666667%; + } + .col-md-9 { + -ms-flex: 0 0 75%; + flex: 0 0 75%; + max-width: 75%; + } + .col-md-10 { + -ms-flex: 0 0 83.333333%; + flex: 0 0 83.333333%; + max-width: 83.333333%; + } + .col-md-11 { + -ms-flex: 0 0 91.666667%; + flex: 0 0 91.666667%; + max-width: 91.666667%; + } + .col-md-12 { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .order-md-first { + -ms-flex-order: -1; + order: -1; + } + .order-md-last { + -ms-flex-order: 13; + order: 13; + } + .order-md-0 { + -ms-flex-order: 0; + order: 0; + } + .order-md-1 { + -ms-flex-order: 1; + order: 1; + } + .order-md-2 { + -ms-flex-order: 2; + order: 2; + } + .order-md-3 { + -ms-flex-order: 3; + order: 3; + } + .order-md-4 { + -ms-flex-order: 4; + order: 4; + } + .order-md-5 { + -ms-flex-order: 5; + order: 5; + } + .order-md-6 { + -ms-flex-order: 6; + order: 6; + } + .order-md-7 { + -ms-flex-order: 7; + order: 7; + } + .order-md-8 { + -ms-flex-order: 8; + order: 8; + } + .order-md-9 { + -ms-flex-order: 9; + order: 9; + } + .order-md-10 { + -ms-flex-order: 10; + order: 10; + } + .order-md-11 { + -ms-flex-order: 11; + order: 11; + } + .order-md-12 { + -ms-flex-order: 12; + order: 12; + } + .offset-md-0 { + margin-left: 0; + } + .offset-md-1 { + margin-left: 8.333333%; + } + .offset-md-2 { + margin-left: 16.666667%; + } + .offset-md-3 { + margin-left: 25%; + } + .offset-md-4 { + margin-left: 33.333333%; + } + .offset-md-5 { + margin-left: 41.666667%; + } + .offset-md-6 { + margin-left: 50%; + } + .offset-md-7 { + margin-left: 58.333333%; + } + .offset-md-8 { + margin-left: 66.666667%; + } + .offset-md-9 { + margin-left: 75%; + } + .offset-md-10 { + margin-left: 83.333333%; + } + .offset-md-11 { + margin-left: 91.666667%; + } +} + +@media (min-width: 992px) { + .col-lg { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + max-width: 100%; + } + .row-cols-lg-1 > * { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .row-cols-lg-2 > * { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .row-cols-lg-3 > * { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .row-cols-lg-4 > * { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .row-cols-lg-5 > * { + -ms-flex: 0 0 20%; + flex: 0 0 20%; + max-width: 20%; + } + .row-cols-lg-6 > * { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-lg-auto { + -ms-flex: 0 0 auto; + flex: 0 0 auto; + width: auto; + max-width: 100%; + } + .col-lg-1 { + -ms-flex: 0 0 8.333333%; + flex: 0 0 8.333333%; + max-width: 8.333333%; + } + .col-lg-2 { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-lg-3 { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .col-lg-4 { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .col-lg-5 { + -ms-flex: 0 0 41.666667%; + flex: 0 0 41.666667%; + max-width: 41.666667%; + } + .col-lg-6 { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .col-lg-7 { + -ms-flex: 0 0 58.333333%; + flex: 0 0 58.333333%; + max-width: 58.333333%; + } + .col-lg-8 { + -ms-flex: 0 0 66.666667%; + flex: 0 0 66.666667%; + max-width: 66.666667%; + } + .col-lg-9 { + -ms-flex: 0 0 75%; + flex: 0 0 75%; + max-width: 75%; + } + .col-lg-10 { + -ms-flex: 0 0 83.333333%; + flex: 0 0 83.333333%; + max-width: 83.333333%; + } + .col-lg-11 { + -ms-flex: 0 0 91.666667%; + flex: 0 0 91.666667%; + max-width: 91.666667%; + } + .col-lg-12 { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .order-lg-first { + -ms-flex-order: -1; + order: -1; + } + .order-lg-last { + -ms-flex-order: 13; + order: 13; + } + .order-lg-0 { + -ms-flex-order: 0; + order: 0; + } + .order-lg-1 { + -ms-flex-order: 1; + order: 1; + } + .order-lg-2 { + -ms-flex-order: 2; + order: 2; + } + .order-lg-3 { + -ms-flex-order: 3; + order: 3; + } + .order-lg-4 { + -ms-flex-order: 4; + order: 4; + } + .order-lg-5 { + -ms-flex-order: 5; + order: 5; + } + .order-lg-6 { + -ms-flex-order: 6; + order: 6; + } + .order-lg-7 { + -ms-flex-order: 7; + order: 7; + } + .order-lg-8 { + -ms-flex-order: 8; + order: 8; + } + .order-lg-9 { + -ms-flex-order: 9; + order: 9; + } + .order-lg-10 { + -ms-flex-order: 10; + order: 10; + } + .order-lg-11 { + -ms-flex-order: 11; + order: 11; + } + .order-lg-12 { + -ms-flex-order: 12; + order: 12; + } + .offset-lg-0 { + margin-left: 0; + } + .offset-lg-1 { + margin-left: 8.333333%; + } + .offset-lg-2 { + margin-left: 16.666667%; + } + .offset-lg-3 { + margin-left: 25%; + } + .offset-lg-4 { + margin-left: 33.333333%; + } + .offset-lg-5 { + margin-left: 41.666667%; + } + .offset-lg-6 { + margin-left: 50%; + } + .offset-lg-7 { + margin-left: 58.333333%; + } + .offset-lg-8 { + margin-left: 66.666667%; + } + .offset-lg-9 { + margin-left: 75%; + } + .offset-lg-10 { + margin-left: 83.333333%; + } + .offset-lg-11 { + margin-left: 91.666667%; + } +} + +@media (min-width: 1200px) { + .col-xl { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + max-width: 100%; + } + .row-cols-xl-1 > * { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .row-cols-xl-2 > * { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .row-cols-xl-3 > * { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .row-cols-xl-4 > * { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .row-cols-xl-5 > * { + -ms-flex: 0 0 20%; + flex: 0 0 20%; + max-width: 20%; + } + .row-cols-xl-6 > * { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-xl-auto { + -ms-flex: 0 0 auto; + flex: 0 0 auto; + width: auto; + max-width: 100%; + } + .col-xl-1 { + -ms-flex: 0 0 8.333333%; + flex: 0 0 8.333333%; + max-width: 8.333333%; + } + .col-xl-2 { + -ms-flex: 0 0 16.666667%; + flex: 0 0 16.666667%; + max-width: 16.666667%; + } + .col-xl-3 { + -ms-flex: 0 0 25%; + flex: 0 0 25%; + max-width: 25%; + } + .col-xl-4 { + -ms-flex: 0 0 33.333333%; + flex: 0 0 33.333333%; + max-width: 33.333333%; + } + .col-xl-5 { + -ms-flex: 0 0 41.666667%; + flex: 0 0 41.666667%; + max-width: 41.666667%; + } + .col-xl-6 { + -ms-flex: 0 0 50%; + flex: 0 0 50%; + max-width: 50%; + } + .col-xl-7 { + -ms-flex: 0 0 58.333333%; + flex: 0 0 58.333333%; + max-width: 58.333333%; + } + .col-xl-8 { + -ms-flex: 0 0 66.666667%; + flex: 0 0 66.666667%; + max-width: 66.666667%; + } + .col-xl-9 { + -ms-flex: 0 0 75%; + flex: 0 0 75%; + max-width: 75%; + } + .col-xl-10 { + -ms-flex: 0 0 83.333333%; + flex: 0 0 83.333333%; + max-width: 83.333333%; + } + .col-xl-11 { + -ms-flex: 0 0 91.666667%; + flex: 0 0 91.666667%; + max-width: 91.666667%; + } + .col-xl-12 { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } + .order-xl-first { + -ms-flex-order: -1; + order: -1; + } + .order-xl-last { + -ms-flex-order: 13; + order: 13; + } + .order-xl-0 { + -ms-flex-order: 0; + order: 0; + } + .order-xl-1 { + -ms-flex-order: 1; + order: 1; + } + .order-xl-2 { + -ms-flex-order: 2; + order: 2; + } + .order-xl-3 { + -ms-flex-order: 3; + order: 3; + } + .order-xl-4 { + -ms-flex-order: 4; + order: 4; + } + .order-xl-5 { + -ms-flex-order: 5; + order: 5; + } + .order-xl-6 { + -ms-flex-order: 6; + order: 6; + } + .order-xl-7 { + -ms-flex-order: 7; + order: 7; + } + .order-xl-8 { + -ms-flex-order: 8; + order: 8; + } + .order-xl-9 { + -ms-flex-order: 9; + order: 9; + } + .order-xl-10 { + -ms-flex-order: 10; + order: 10; + } + .order-xl-11 { + -ms-flex-order: 11; + order: 11; + } + .order-xl-12 { + -ms-flex-order: 12; + order: 12; + } + .offset-xl-0 { + margin-left: 0; + } + .offset-xl-1 { + margin-left: 8.333333%; + } + .offset-xl-2 { + margin-left: 16.666667%; + } + .offset-xl-3 { + margin-left: 25%; + } + .offset-xl-4 { + margin-left: 33.333333%; + } + .offset-xl-5 { + margin-left: 41.666667%; + } + .offset-xl-6 { + margin-left: 50%; + } + .offset-xl-7 { + margin-left: 58.333333%; + } + .offset-xl-8 { + margin-left: 66.666667%; + } + .offset-xl-9 { + margin-left: 75%; + } + .offset-xl-10 { + margin-left: 83.333333%; + } + .offset-xl-11 { + margin-left: 91.666667%; + } +} + +.table { + width: 100%; + margin-bottom: 1rem; + color: #212529; +} + +.table th, +.table td { + padding: 0.75rem; + vertical-align: top; + border-top: 1px solid #dee2e6; +} + +.table thead th { + vertical-align: bottom; + border-bottom: 2px solid #dee2e6; +} + +.table tbody + tbody { + border-top: 2px solid #dee2e6; +} + +.table-sm th, +.table-sm td { + padding: 0.3rem; +} + +.table-bordered { + border: 1px solid #dee2e6; +} + +.table-bordered th, +.table-bordered td { + border: 1px solid #dee2e6; +} + +.table-bordered thead th, +.table-bordered thead td { + border-bottom-width: 2px; +} + +.table-borderless th, +.table-borderless td, +.table-borderless thead th, +.table-borderless tbody + tbody { + border: 0; +} + +.table-striped tbody tr:nth-of-type(odd) { + background-color: rgba(0, 0, 0, 0.05); +} + +.table-hover tbody tr:hover { + color: #212529; + background-color: rgba(0, 0, 0, 0.075); +} + +.table-primary, +.table-primary > th, +.table-primary > td { + background-color: #b8daff; +} + +.table-primary th, +.table-primary td, +.table-primary thead th, +.table-primary tbody + tbody { + border-color: #7abaff; +} + +.table-hover .table-primary:hover { + background-color: #9fcdff; +} + +.table-hover .table-primary:hover > td, +.table-hover .table-primary:hover > th { + background-color: #9fcdff; +} + +.table-secondary, +.table-secondary > th, +.table-secondary > td { + background-color: #d6d8db; +} + +.table-secondary th, +.table-secondary td, +.table-secondary thead th, +.table-secondary tbody + tbody { + border-color: #b3b7bb; +} + +.table-hover .table-secondary:hover { + background-color: #c8cbcf; +} + +.table-hover .table-secondary:hover > td, +.table-hover .table-secondary:hover > th { + background-color: #c8cbcf; +} + +.table-success, +.table-success > th, +.table-success > td { + background-color: #c3e6cb; +} + +.table-success th, +.table-success td, +.table-success thead th, +.table-success tbody + tbody { + border-color: #8fd19e; +} + +.table-hover .table-success:hover { + background-color: #b1dfbb; +} + +.table-hover .table-success:hover > td, +.table-hover .table-success:hover > th { + background-color: #b1dfbb; +} + +.table-info, +.table-info > th, +.table-info > td { + background-color: #bee5eb; +} + +.table-info th, +.table-info td, +.table-info thead th, +.table-info tbody + tbody { + border-color: #86cfda; +} + +.table-hover .table-info:hover { + background-color: #abdde5; +} + +.table-hover .table-info:hover > td, +.table-hover .table-info:hover > th { + background-color: #abdde5; +} + +.table-warning, +.table-warning > th, +.table-warning > td { + background-color: #ffeeba; +} + +.table-warning th, +.table-warning td, +.table-warning thead th, +.table-warning tbody + tbody { + border-color: #ffdf7e; +} + +.table-hover .table-warning:hover { + background-color: #ffe8a1; +} + +.table-hover .table-warning:hover > td, +.table-hover .table-warning:hover > th { + background-color: #ffe8a1; +} + +.table-danger, +.table-danger > th, +.table-danger > td { + background-color: #f5c6cb; +} + +.table-danger th, +.table-danger td, +.table-danger thead th, +.table-danger tbody + tbody { + border-color: #ed969e; +} + +.table-hover .table-danger:hover { + background-color: #f1b0b7; +} + +.table-hover .table-danger:hover > td, +.table-hover .table-danger:hover > th { + background-color: #f1b0b7; +} + +.table-light, +.table-light > th, +.table-light > td { + background-color: #fdfdfe; +} + +.table-light th, +.table-light td, +.table-light thead th, +.table-light tbody + tbody { + border-color: #fbfcfc; +} + +.table-hover .table-light:hover { + background-color: #ececf6; +} + +.table-hover .table-light:hover > td, +.table-hover .table-light:hover > th { + background-color: #ececf6; +} + +.table-dark, +.table-dark > th, +.table-dark > td { + background-color: #c6c8ca; +} + +.table-dark th, +.table-dark td, +.table-dark thead th, +.table-dark tbody + tbody { + border-color: #95999c; +} + +.table-hover .table-dark:hover { + background-color: #b9bbbe; +} + +.table-hover .table-dark:hover > td, +.table-hover .table-dark:hover > th { + background-color: #b9bbbe; +} + +.table-active, +.table-active > th, +.table-active > td { + background-color: rgba(0, 0, 0, 0.075); +} + +.table-hover .table-active:hover { + background-color: rgba(0, 0, 0, 0.075); +} + +.table-hover .table-active:hover > td, +.table-hover .table-active:hover > th { + background-color: rgba(0, 0, 0, 0.075); +} + +.table .thead-dark th { + color: #fff; + background-color: #343a40; + border-color: #454d55; +} + +.table .thead-light th { + color: #495057; + background-color: #e9ecef; + border-color: #dee2e6; +} + +.table-dark { + color: #fff; + background-color: #343a40; +} + +.table-dark th, +.table-dark td, +.table-dark thead th { + border-color: #454d55; +} + +.table-dark.table-bordered { + border: 0; +} + +.table-dark.table-striped tbody tr:nth-of-type(odd) { + background-color: rgba(255, 255, 255, 0.05); +} + +.table-dark.table-hover tbody tr:hover { + color: #fff; + background-color: rgba(255, 255, 255, 0.075); +} + +@media (max-width: 575.98px) { + .table-responsive-sm { + display: block; + width: 100%; + overflow-x: auto; + -webkit-overflow-scrolling: touch; + } + .table-responsive-sm > .table-bordered { + border: 0; + } +} + +@media (max-width: 767.98px) { + .table-responsive-md { + display: block; + width: 100%; + overflow-x: auto; + -webkit-overflow-scrolling: touch; + } + .table-responsive-md > .table-bordered { + border: 0; + } +} + +@media (max-width: 991.98px) { + .table-responsive-lg { + display: block; + width: 100%; + overflow-x: auto; + -webkit-overflow-scrolling: touch; + } + .table-responsive-lg > .table-bordered { + border: 0; + } +} + +@media (max-width: 1199.98px) { + .table-responsive-xl { + display: block; + width: 100%; + overflow-x: auto; + -webkit-overflow-scrolling: touch; + } + .table-responsive-xl > .table-bordered { + border: 0; + } +} + +.table-responsive { + display: block; + width: 100%; + overflow-x: auto; + -webkit-overflow-scrolling: touch; +} + +.table-responsive > .table-bordered { + border: 0; +} + +.form-control { + display: block; + width: 100%; + height: calc(1.5em + 0.75rem + 2px); + padding: 0.375rem 0.75rem; + font-size: 1rem; + font-weight: 400; + line-height: 1.5; + color: #495057; + background-color: #fff; + background-clip: padding-box; + border: 1px solid #ced4da; + border-radius: 0.25rem; + transition: border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .form-control { + transition: none; + } +} + +.form-control::-ms-expand { + background-color: transparent; + border: 0; +} + +.form-control:-moz-focusring { + color: transparent; + text-shadow: 0 0 0 #495057; +} + +.form-control:focus { + color: #495057; + background-color: #fff; + border-color: #80bdff; + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.form-control::-webkit-input-placeholder { + color: #6c757d; + opacity: 1; +} + +.form-control::-moz-placeholder { + color: #6c757d; + opacity: 1; +} + +.form-control:-ms-input-placeholder { + color: #6c757d; + opacity: 1; +} + +.form-control::-ms-input-placeholder { + color: #6c757d; + opacity: 1; +} + +.form-control::placeholder { + color: #6c757d; + opacity: 1; +} + +.form-control:disabled, .form-control[readonly] { + background-color: #e9ecef; + opacity: 1; +} + +select.form-control:focus::-ms-value { + color: #495057; + background-color: #fff; +} + +.form-control-file, +.form-control-range { + display: block; + width: 100%; +} + +.col-form-label { + padding-top: calc(0.375rem + 1px); + padding-bottom: calc(0.375rem + 1px); + margin-bottom: 0; + font-size: inherit; + line-height: 1.5; +} + +.col-form-label-lg { + padding-top: calc(0.5rem + 1px); + padding-bottom: calc(0.5rem + 1px); + font-size: 1.25rem; + line-height: 1.5; +} + +.col-form-label-sm { + padding-top: calc(0.25rem + 1px); + padding-bottom: calc(0.25rem + 1px); + font-size: 0.875rem; + line-height: 1.5; +} + +.form-control-plaintext { + display: block; + width: 100%; + padding: 0.375rem 0; + margin-bottom: 0; + font-size: 1rem; + line-height: 1.5; + color: #212529; + background-color: transparent; + border: solid transparent; + border-width: 1px 0; +} + +.form-control-plaintext.form-control-sm, .form-control-plaintext.form-control-lg { + padding-right: 0; + padding-left: 0; +} + +.form-control-sm { + height: calc(1.5em + 0.5rem + 2px); + padding: 0.25rem 0.5rem; + font-size: 0.875rem; + line-height: 1.5; + border-radius: 0.2rem; +} + +.form-control-lg { + height: calc(1.5em + 1rem + 2px); + padding: 0.5rem 1rem; + font-size: 1.25rem; + line-height: 1.5; + border-radius: 0.3rem; +} + +select.form-control[size], select.form-control[multiple] { + height: auto; +} + +textarea.form-control { + height: auto; +} + +.form-group { + margin-bottom: 1rem; +} + +.form-text { + display: block; + margin-top: 0.25rem; +} + +.form-row { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + margin-right: -5px; + margin-left: -5px; +} + +.form-row > .col, +.form-row > [class*="col-"] { + padding-right: 5px; + padding-left: 5px; +} + +.form-check { + position: relative; + display: block; + padding-left: 1.25rem; +} + +.form-check-input { + position: absolute; + margin-top: 0.3rem; + margin-left: -1.25rem; +} + +.form-check-input[disabled] ~ .form-check-label, +.form-check-input:disabled ~ .form-check-label { + color: #6c757d; +} + +.form-check-label { + margin-bottom: 0; +} + +.form-check-inline { + display: -ms-inline-flexbox; + display: inline-flex; + -ms-flex-align: center; + align-items: center; + padding-left: 0; + margin-right: 0.75rem; +} + +.form-check-inline .form-check-input { + position: static; + margin-top: 0; + margin-right: 0.3125rem; + margin-left: 0; +} + +.valid-feedback { + display: none; + width: 100%; + margin-top: 0.25rem; + font-size: 80%; + color: #28a745; +} + +.valid-tooltip { + position: absolute; + top: 100%; + z-index: 5; + display: none; + max-width: 100%; + padding: 0.25rem 0.5rem; + margin-top: .1rem; + font-size: 0.875rem; + line-height: 1.5; + color: #fff; + background-color: rgba(40, 167, 69, 0.9); + border-radius: 0.25rem; +} + +.was-validated :valid ~ .valid-feedback, +.was-validated :valid ~ .valid-tooltip, +.is-valid ~ .valid-feedback, +.is-valid ~ .valid-tooltip { + display: block; +} + +.was-validated .form-control:valid, .form-control.is-valid { + border-color: #28a745; + padding-right: calc(1.5em + 0.75rem); + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3e%3c/svg%3e"); + background-repeat: no-repeat; + background-position: right calc(0.375em + 0.1875rem) center; + background-size: calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); +} + +.was-validated .form-control:valid:focus, .form-control.is-valid:focus { + border-color: #28a745; + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); +} + +.was-validated textarea.form-control:valid, textarea.form-control.is-valid { + padding-right: calc(1.5em + 0.75rem); + background-position: top calc(0.375em + 0.1875rem) right calc(0.375em + 0.1875rem); +} + +.was-validated .custom-select:valid, .custom-select.is-valid { + border-color: #28a745; + padding-right: calc(0.75em + 2.3125rem); + background: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px, url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3e%3c/svg%3e") #fff no-repeat center right 1.75rem/calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); +} + +.was-validated .custom-select:valid:focus, .custom-select.is-valid:focus { + border-color: #28a745; + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); +} + +.was-validated .form-check-input:valid ~ .form-check-label, .form-check-input.is-valid ~ .form-check-label { + color: #28a745; +} + +.was-validated .form-check-input:valid ~ .valid-feedback, +.was-validated .form-check-input:valid ~ .valid-tooltip, .form-check-input.is-valid ~ .valid-feedback, +.form-check-input.is-valid ~ .valid-tooltip { + display: block; +} + +.was-validated .custom-control-input:valid ~ .custom-control-label, .custom-control-input.is-valid ~ .custom-control-label { + color: #28a745; +} + +.was-validated .custom-control-input:valid ~ .custom-control-label::before, .custom-control-input.is-valid ~ .custom-control-label::before { + border-color: #28a745; +} + +.was-validated .custom-control-input:valid:checked ~ .custom-control-label::before, .custom-control-input.is-valid:checked ~ .custom-control-label::before { + border-color: #34ce57; + background-color: #34ce57; +} + +.was-validated .custom-control-input:valid:focus ~ .custom-control-label::before, .custom-control-input.is-valid:focus ~ .custom-control-label::before { + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); +} + +.was-validated .custom-control-input:valid:focus:not(:checked) ~ .custom-control-label::before, .custom-control-input.is-valid:focus:not(:checked) ~ .custom-control-label::before { + border-color: #28a745; +} + +.was-validated .custom-file-input:valid ~ .custom-file-label, .custom-file-input.is-valid ~ .custom-file-label { + border-color: #28a745; +} + +.was-validated .custom-file-input:valid:focus ~ .custom-file-label, .custom-file-input.is-valid:focus ~ .custom-file-label { + border-color: #28a745; + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); +} + +.invalid-feedback { + display: none; + width: 100%; + margin-top: 0.25rem; + font-size: 80%; + color: #dc3545; +} + +.invalid-tooltip { + position: absolute; + top: 100%; + z-index: 5; + display: none; + max-width: 100%; + padding: 0.25rem 0.5rem; + margin-top: .1rem; + font-size: 0.875rem; + line-height: 1.5; + color: #fff; + background-color: rgba(220, 53, 69, 0.9); + border-radius: 0.25rem; +} + +.was-validated :invalid ~ .invalid-feedback, +.was-validated :invalid ~ .invalid-tooltip, +.is-invalid ~ .invalid-feedback, +.is-invalid ~ .invalid-tooltip { + display: block; +} + +.was-validated .form-control:invalid, .form-control.is-invalid { + border-color: #dc3545; + padding-right: calc(1.5em + 0.75rem); + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545' viewBox='0 0 12 12'%3e%3ccircle cx='6' cy='6' r='4.5'/%3e%3cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3e%3ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3e%3c/svg%3e"); + background-repeat: no-repeat; + background-position: right calc(0.375em + 0.1875rem) center; + background-size: calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); +} + +.was-validated .form-control:invalid:focus, .form-control.is-invalid:focus { + border-color: #dc3545; + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); +} + +.was-validated textarea.form-control:invalid, textarea.form-control.is-invalid { + padding-right: calc(1.5em + 0.75rem); + background-position: top calc(0.375em + 0.1875rem) right calc(0.375em + 0.1875rem); +} + +.was-validated .custom-select:invalid, .custom-select.is-invalid { + border-color: #dc3545; + padding-right: calc(0.75em + 2.3125rem); + background: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px, url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545' viewBox='0 0 12 12'%3e%3ccircle cx='6' cy='6' r='4.5'/%3e%3cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3e%3ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3e%3c/svg%3e") #fff no-repeat center right 1.75rem/calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); +} + +.was-validated .custom-select:invalid:focus, .custom-select.is-invalid:focus { + border-color: #dc3545; + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); +} + +.was-validated .form-check-input:invalid ~ .form-check-label, .form-check-input.is-invalid ~ .form-check-label { + color: #dc3545; +} + +.was-validated .form-check-input:invalid ~ .invalid-feedback, +.was-validated .form-check-input:invalid ~ .invalid-tooltip, .form-check-input.is-invalid ~ .invalid-feedback, +.form-check-input.is-invalid ~ .invalid-tooltip { + display: block; +} + +.was-validated .custom-control-input:invalid ~ .custom-control-label, .custom-control-input.is-invalid ~ .custom-control-label { + color: #dc3545; +} + +.was-validated .custom-control-input:invalid ~ .custom-control-label::before, .custom-control-input.is-invalid ~ .custom-control-label::before { + border-color: #dc3545; +} + +.was-validated .custom-control-input:invalid:checked ~ .custom-control-label::before, .custom-control-input.is-invalid:checked ~ .custom-control-label::before { + border-color: #e4606d; + background-color: #e4606d; +} + +.was-validated .custom-control-input:invalid:focus ~ .custom-control-label::before, .custom-control-input.is-invalid:focus ~ .custom-control-label::before { + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); +} + +.was-validated .custom-control-input:invalid:focus:not(:checked) ~ .custom-control-label::before, .custom-control-input.is-invalid:focus:not(:checked) ~ .custom-control-label::before { + border-color: #dc3545; +} + +.was-validated .custom-file-input:invalid ~ .custom-file-label, .custom-file-input.is-invalid ~ .custom-file-label { + border-color: #dc3545; +} + +.was-validated .custom-file-input:invalid:focus ~ .custom-file-label, .custom-file-input.is-invalid:focus ~ .custom-file-label { + border-color: #dc3545; + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); +} + +.form-inline { + display: -ms-flexbox; + display: flex; + -ms-flex-flow: row wrap; + flex-flow: row wrap; + -ms-flex-align: center; + align-items: center; +} + +.form-inline .form-check { + width: 100%; +} + +@media (min-width: 576px) { + .form-inline label { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: center; + justify-content: center; + margin-bottom: 0; + } + .form-inline .form-group { + display: -ms-flexbox; + display: flex; + -ms-flex: 0 0 auto; + flex: 0 0 auto; + -ms-flex-flow: row wrap; + flex-flow: row wrap; + -ms-flex-align: center; + align-items: center; + margin-bottom: 0; + } + .form-inline .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .form-inline .form-control-plaintext { + display: inline-block; + } + .form-inline .input-group, + .form-inline .custom-select { + width: auto; + } + .form-inline .form-check { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: center; + justify-content: center; + width: auto; + padding-left: 0; + } + .form-inline .form-check-input { + position: relative; + -ms-flex-negative: 0; + flex-shrink: 0; + margin-top: 0; + margin-right: 0.25rem; + margin-left: 0; + } + .form-inline .custom-control { + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: center; + justify-content: center; + } + .form-inline .custom-control-label { + margin-bottom: 0; + } +} + +.btn { + display: inline-block; + font-weight: 400; + color: #212529; + text-align: center; + vertical-align: middle; + cursor: pointer; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + background-color: transparent; + border: 1px solid transparent; + padding: 0.375rem 0.75rem; + font-size: 1rem; + line-height: 1.5; + border-radius: 0.25rem; + transition: color 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .btn { + transition: none; + } +} + +.btn:hover { + color: #212529; + text-decoration: none; +} + +.btn:focus, .btn.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.btn.disabled, .btn:disabled { + opacity: 0.65; +} + +a.btn.disabled, +fieldset:disabled a.btn { + pointer-events: none; +} + +.btn-primary { + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.btn-primary:hover { + color: #fff; + background-color: #0069d9; + border-color: #0062cc; +} + +.btn-primary:focus, .btn-primary.focus { + color: #fff; + background-color: #0069d9; + border-color: #0062cc; + box-shadow: 0 0 0 0.2rem rgba(38, 143, 255, 0.5); +} + +.btn-primary.disabled, .btn-primary:disabled { + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.btn-primary:not(:disabled):not(.disabled):active, .btn-primary:not(:disabled):not(.disabled).active, +.show > .btn-primary.dropdown-toggle { + color: #fff; + background-color: #0062cc; + border-color: #005cbf; +} + +.btn-primary:not(:disabled):not(.disabled):active:focus, .btn-primary:not(:disabled):not(.disabled).active:focus, +.show > .btn-primary.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(38, 143, 255, 0.5); +} + +.btn-secondary { + color: #fff; + background-color: #6c757d; + border-color: #6c757d; +} + +.btn-secondary:hover { + color: #fff; + background-color: #5a6268; + border-color: #545b62; +} + +.btn-secondary:focus, .btn-secondary.focus { + color: #fff; + background-color: #5a6268; + border-color: #545b62; + box-shadow: 0 0 0 0.2rem rgba(130, 138, 145, 0.5); +} + +.btn-secondary.disabled, .btn-secondary:disabled { + color: #fff; + background-color: #6c757d; + border-color: #6c757d; +} + +.btn-secondary:not(:disabled):not(.disabled):active, .btn-secondary:not(:disabled):not(.disabled).active, +.show > .btn-secondary.dropdown-toggle { + color: #fff; + background-color: #545b62; + border-color: #4e555b; +} + +.btn-secondary:not(:disabled):not(.disabled):active:focus, .btn-secondary:not(:disabled):not(.disabled).active:focus, +.show > .btn-secondary.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(130, 138, 145, 0.5); +} + +.btn-success { + color: #fff; + background-color: #28a745; + border-color: #28a745; +} + +.btn-success:hover { + color: #fff; + background-color: #218838; + border-color: #1e7e34; +} + +.btn-success:focus, .btn-success.focus { + color: #fff; + background-color: #218838; + border-color: #1e7e34; + box-shadow: 0 0 0 0.2rem rgba(72, 180, 97, 0.5); +} + +.btn-success.disabled, .btn-success:disabled { + color: #fff; + background-color: #28a745; + border-color: #28a745; +} + +.btn-success:not(:disabled):not(.disabled):active, .btn-success:not(:disabled):not(.disabled).active, +.show > .btn-success.dropdown-toggle { + color: #fff; + background-color: #1e7e34; + border-color: #1c7430; +} + +.btn-success:not(:disabled):not(.disabled):active:focus, .btn-success:not(:disabled):not(.disabled).active:focus, +.show > .btn-success.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(72, 180, 97, 0.5); +} + +.btn-info { + color: #fff; + background-color: #17a2b8; + border-color: #17a2b8; +} + +.btn-info:hover { + color: #fff; + background-color: #138496; + border-color: #117a8b; +} + +.btn-info:focus, .btn-info.focus { + color: #fff; + background-color: #138496; + border-color: #117a8b; + box-shadow: 0 0 0 0.2rem rgba(58, 176, 195, 0.5); +} + +.btn-info.disabled, .btn-info:disabled { + color: #fff; + background-color: #17a2b8; + border-color: #17a2b8; +} + +.btn-info:not(:disabled):not(.disabled):active, .btn-info:not(:disabled):not(.disabled).active, +.show > .btn-info.dropdown-toggle { + color: #fff; + background-color: #117a8b; + border-color: #10707f; +} + +.btn-info:not(:disabled):not(.disabled):active:focus, .btn-info:not(:disabled):not(.disabled).active:focus, +.show > .btn-info.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(58, 176, 195, 0.5); +} + +.btn-warning { + color: #212529; + background-color: #ffc107; + border-color: #ffc107; +} + +.btn-warning:hover { + color: #212529; + background-color: #e0a800; + border-color: #d39e00; +} + +.btn-warning:focus, .btn-warning.focus { + color: #212529; + background-color: #e0a800; + border-color: #d39e00; + box-shadow: 0 0 0 0.2rem rgba(222, 170, 12, 0.5); +} + +.btn-warning.disabled, .btn-warning:disabled { + color: #212529; + background-color: #ffc107; + border-color: #ffc107; +} + +.btn-warning:not(:disabled):not(.disabled):active, .btn-warning:not(:disabled):not(.disabled).active, +.show > .btn-warning.dropdown-toggle { + color: #212529; + background-color: #d39e00; + border-color: #c69500; +} + +.btn-warning:not(:disabled):not(.disabled):active:focus, .btn-warning:not(:disabled):not(.disabled).active:focus, +.show > .btn-warning.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(222, 170, 12, 0.5); +} + +.btn-danger { + color: #fff; + background-color: #dc3545; + border-color: #dc3545; +} + +.btn-danger:hover { + color: #fff; + background-color: #c82333; + border-color: #bd2130; +} + +.btn-danger:focus, .btn-danger.focus { + color: #fff; + background-color: #c82333; + border-color: #bd2130; + box-shadow: 0 0 0 0.2rem rgba(225, 83, 97, 0.5); +} + +.btn-danger.disabled, .btn-danger:disabled { + color: #fff; + background-color: #dc3545; + border-color: #dc3545; +} + +.btn-danger:not(:disabled):not(.disabled):active, .btn-danger:not(:disabled):not(.disabled).active, +.show > .btn-danger.dropdown-toggle { + color: #fff; + background-color: #bd2130; + border-color: #b21f2d; +} + +.btn-danger:not(:disabled):not(.disabled):active:focus, .btn-danger:not(:disabled):not(.disabled).active:focus, +.show > .btn-danger.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(225, 83, 97, 0.5); +} + +.btn-light { + color: #212529; + background-color: #f8f9fa; + border-color: #f8f9fa; +} + +.btn-light:hover { + color: #212529; + background-color: #e2e6ea; + border-color: #dae0e5; +} + +.btn-light:focus, .btn-light.focus { + color: #212529; + background-color: #e2e6ea; + border-color: #dae0e5; + box-shadow: 0 0 0 0.2rem rgba(216, 217, 219, 0.5); +} + +.btn-light.disabled, .btn-light:disabled { + color: #212529; + background-color: #f8f9fa; + border-color: #f8f9fa; +} + +.btn-light:not(:disabled):not(.disabled):active, .btn-light:not(:disabled):not(.disabled).active, +.show > .btn-light.dropdown-toggle { + color: #212529; + background-color: #dae0e5; + border-color: #d3d9df; +} + +.btn-light:not(:disabled):not(.disabled):active:focus, .btn-light:not(:disabled):not(.disabled).active:focus, +.show > .btn-light.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(216, 217, 219, 0.5); +} + +.btn-dark { + color: #fff; + background-color: #343a40; + border-color: #343a40; +} + +.btn-dark:hover { + color: #fff; + background-color: #23272b; + border-color: #1d2124; +} + +.btn-dark:focus, .btn-dark.focus { + color: #fff; + background-color: #23272b; + border-color: #1d2124; + box-shadow: 0 0 0 0.2rem rgba(82, 88, 93, 0.5); +} + +.btn-dark.disabled, .btn-dark:disabled { + color: #fff; + background-color: #343a40; + border-color: #343a40; +} + +.btn-dark:not(:disabled):not(.disabled):active, .btn-dark:not(:disabled):not(.disabled).active, +.show > .btn-dark.dropdown-toggle { + color: #fff; + background-color: #1d2124; + border-color: #171a1d; +} + +.btn-dark:not(:disabled):not(.disabled):active:focus, .btn-dark:not(:disabled):not(.disabled).active:focus, +.show > .btn-dark.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(82, 88, 93, 0.5); +} + +.btn-outline-primary { + color: #007bff; + border-color: #007bff; +} + +.btn-outline-primary:hover { + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.btn-outline-primary:focus, .btn-outline-primary.focus { + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.5); +} + +.btn-outline-primary.disabled, .btn-outline-primary:disabled { + color: #007bff; + background-color: transparent; +} + +.btn-outline-primary:not(:disabled):not(.disabled):active, .btn-outline-primary:not(:disabled):not(.disabled).active, +.show > .btn-outline-primary.dropdown-toggle { + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.btn-outline-primary:not(:disabled):not(.disabled):active:focus, .btn-outline-primary:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-primary.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.5); +} + +.btn-outline-secondary { + color: #6c757d; + border-color: #6c757d; +} + +.btn-outline-secondary:hover { + color: #fff; + background-color: #6c757d; + border-color: #6c757d; +} + +.btn-outline-secondary:focus, .btn-outline-secondary.focus { + box-shadow: 0 0 0 0.2rem rgba(108, 117, 125, 0.5); +} + +.btn-outline-secondary.disabled, .btn-outline-secondary:disabled { + color: #6c757d; + background-color: transparent; +} + +.btn-outline-secondary:not(:disabled):not(.disabled):active, .btn-outline-secondary:not(:disabled):not(.disabled).active, +.show > .btn-outline-secondary.dropdown-toggle { + color: #fff; + background-color: #6c757d; + border-color: #6c757d; +} + +.btn-outline-secondary:not(:disabled):not(.disabled):active:focus, .btn-outline-secondary:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-secondary.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(108, 117, 125, 0.5); +} + +.btn-outline-success { + color: #28a745; + border-color: #28a745; +} + +.btn-outline-success:hover { + color: #fff; + background-color: #28a745; + border-color: #28a745; +} + +.btn-outline-success:focus, .btn-outline-success.focus { + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.5); +} + +.btn-outline-success.disabled, .btn-outline-success:disabled { + color: #28a745; + background-color: transparent; +} + +.btn-outline-success:not(:disabled):not(.disabled):active, .btn-outline-success:not(:disabled):not(.disabled).active, +.show > .btn-outline-success.dropdown-toggle { + color: #fff; + background-color: #28a745; + border-color: #28a745; +} + +.btn-outline-success:not(:disabled):not(.disabled):active:focus, .btn-outline-success:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-success.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.5); +} + +.btn-outline-info { + color: #17a2b8; + border-color: #17a2b8; +} + +.btn-outline-info:hover { + color: #fff; + background-color: #17a2b8; + border-color: #17a2b8; +} + +.btn-outline-info:focus, .btn-outline-info.focus { + box-shadow: 0 0 0 0.2rem rgba(23, 162, 184, 0.5); +} + +.btn-outline-info.disabled, .btn-outline-info:disabled { + color: #17a2b8; + background-color: transparent; +} + +.btn-outline-info:not(:disabled):not(.disabled):active, .btn-outline-info:not(:disabled):not(.disabled).active, +.show > .btn-outline-info.dropdown-toggle { + color: #fff; + background-color: #17a2b8; + border-color: #17a2b8; +} + +.btn-outline-info:not(:disabled):not(.disabled):active:focus, .btn-outline-info:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-info.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(23, 162, 184, 0.5); +} + +.btn-outline-warning { + color: #ffc107; + border-color: #ffc107; +} + +.btn-outline-warning:hover { + color: #212529; + background-color: #ffc107; + border-color: #ffc107; +} + +.btn-outline-warning:focus, .btn-outline-warning.focus { + box-shadow: 0 0 0 0.2rem rgba(255, 193, 7, 0.5); +} + +.btn-outline-warning.disabled, .btn-outline-warning:disabled { + color: #ffc107; + background-color: transparent; +} + +.btn-outline-warning:not(:disabled):not(.disabled):active, .btn-outline-warning:not(:disabled):not(.disabled).active, +.show > .btn-outline-warning.dropdown-toggle { + color: #212529; + background-color: #ffc107; + border-color: #ffc107; +} + +.btn-outline-warning:not(:disabled):not(.disabled):active:focus, .btn-outline-warning:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-warning.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(255, 193, 7, 0.5); +} + +.btn-outline-danger { + color: #dc3545; + border-color: #dc3545; +} + +.btn-outline-danger:hover { + color: #fff; + background-color: #dc3545; + border-color: #dc3545; +} + +.btn-outline-danger:focus, .btn-outline-danger.focus { + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.5); +} + +.btn-outline-danger.disabled, .btn-outline-danger:disabled { + color: #dc3545; + background-color: transparent; +} + +.btn-outline-danger:not(:disabled):not(.disabled):active, .btn-outline-danger:not(:disabled):not(.disabled).active, +.show > .btn-outline-danger.dropdown-toggle { + color: #fff; + background-color: #dc3545; + border-color: #dc3545; +} + +.btn-outline-danger:not(:disabled):not(.disabled):active:focus, .btn-outline-danger:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-danger.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.5); +} + +.btn-outline-light { + color: #f8f9fa; + border-color: #f8f9fa; +} + +.btn-outline-light:hover { + color: #212529; + background-color: #f8f9fa; + border-color: #f8f9fa; +} + +.btn-outline-light:focus, .btn-outline-light.focus { + box-shadow: 0 0 0 0.2rem rgba(248, 249, 250, 0.5); +} + +.btn-outline-light.disabled, .btn-outline-light:disabled { + color: #f8f9fa; + background-color: transparent; +} + +.btn-outline-light:not(:disabled):not(.disabled):active, .btn-outline-light:not(:disabled):not(.disabled).active, +.show > .btn-outline-light.dropdown-toggle { + color: #212529; + background-color: #f8f9fa; + border-color: #f8f9fa; +} + +.btn-outline-light:not(:disabled):not(.disabled):active:focus, .btn-outline-light:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-light.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(248, 249, 250, 0.5); +} + +.btn-outline-dark { + color: #343a40; + border-color: #343a40; +} + +.btn-outline-dark:hover { + color: #fff; + background-color: #343a40; + border-color: #343a40; +} + +.btn-outline-dark:focus, .btn-outline-dark.focus { + box-shadow: 0 0 0 0.2rem rgba(52, 58, 64, 0.5); +} + +.btn-outline-dark.disabled, .btn-outline-dark:disabled { + color: #343a40; + background-color: transparent; +} + +.btn-outline-dark:not(:disabled):not(.disabled):active, .btn-outline-dark:not(:disabled):not(.disabled).active, +.show > .btn-outline-dark.dropdown-toggle { + color: #fff; + background-color: #343a40; + border-color: #343a40; +} + +.btn-outline-dark:not(:disabled):not(.disabled):active:focus, .btn-outline-dark:not(:disabled):not(.disabled).active:focus, +.show > .btn-outline-dark.dropdown-toggle:focus { + box-shadow: 0 0 0 0.2rem rgba(52, 58, 64, 0.5); +} + +.btn-link { + font-weight: 400; + color: #007bff; + text-decoration: none; +} + +.btn-link:hover { + color: #0056b3; + text-decoration: underline; +} + +.btn-link:focus, .btn-link.focus { + text-decoration: underline; + box-shadow: none; +} + +.btn-link:disabled, .btn-link.disabled { + color: #6c757d; + pointer-events: none; +} + +.btn-lg, .btn-group-lg > .btn { + padding: 0.5rem 1rem; + font-size: 1.25rem; + line-height: 1.5; + border-radius: 0.3rem; +} + +.btn-sm, .btn-group-sm > .btn { + padding: 0.25rem 0.5rem; + font-size: 0.875rem; + line-height: 1.5; + border-radius: 0.2rem; +} + +.btn-block { + display: block; + width: 100%; +} + +.btn-block + .btn-block { + margin-top: 0.5rem; +} + +input[type="submit"].btn-block, +input[type="reset"].btn-block, +input[type="button"].btn-block { + width: 100%; +} + +.fade { + transition: opacity 0.15s linear; +} + +@media (prefers-reduced-motion: reduce) { + .fade { + transition: none; + } +} + +.fade:not(.show) { + opacity: 0; +} + +.collapse:not(.show) { + display: none; +} + +.collapsing { + position: relative; + height: 0; + overflow: hidden; + transition: height 0.35s ease; +} + +@media (prefers-reduced-motion: reduce) { + .collapsing { + transition: none; + } +} + +.dropup, +.dropright, +.dropdown, +.dropleft { + position: relative; +} + +.dropdown-toggle { + white-space: nowrap; +} + +.dropdown-toggle::after { + display: inline-block; + margin-left: 0.255em; + vertical-align: 0.255em; + content: ""; + border-top: 0.3em solid; + border-right: 0.3em solid transparent; + border-bottom: 0; + border-left: 0.3em solid transparent; +} + +.dropdown-toggle:empty::after { + margin-left: 0; +} + +.dropdown-menu { + position: absolute; + top: 100%; + left: 0; + z-index: 1000; + display: none; + float: left; + min-width: 10rem; + padding: 0.5rem 0; + margin: 0.125rem 0 0; + font-size: 1rem; + color: #212529; + text-align: left; + list-style: none; + background-color: #fff; + background-clip: padding-box; + border: 1px solid rgba(0, 0, 0, 0.15); + border-radius: 0.25rem; +} + +.dropdown-menu-left { + right: auto; + left: 0; +} + +.dropdown-menu-right { + right: 0; + left: auto; +} + +@media (min-width: 576px) { + .dropdown-menu-sm-left { + right: auto; + left: 0; + } + .dropdown-menu-sm-right { + right: 0; + left: auto; + } +} + +@media (min-width: 768px) { + .dropdown-menu-md-left { + right: auto; + left: 0; + } + .dropdown-menu-md-right { + right: 0; + left: auto; + } +} + +@media (min-width: 992px) { + .dropdown-menu-lg-left { + right: auto; + left: 0; + } + .dropdown-menu-lg-right { + right: 0; + left: auto; + } +} + +@media (min-width: 1200px) { + .dropdown-menu-xl-left { + right: auto; + left: 0; + } + .dropdown-menu-xl-right { + right: 0; + left: auto; + } +} + +.dropup .dropdown-menu { + top: auto; + bottom: 100%; + margin-top: 0; + margin-bottom: 0.125rem; +} + +.dropup .dropdown-toggle::after { + display: inline-block; + margin-left: 0.255em; + vertical-align: 0.255em; + content: ""; + border-top: 0; + border-right: 0.3em solid transparent; + border-bottom: 0.3em solid; + border-left: 0.3em solid transparent; +} + +.dropup .dropdown-toggle:empty::after { + margin-left: 0; +} + +.dropright .dropdown-menu { + top: 0; + right: auto; + left: 100%; + margin-top: 0; + margin-left: 0.125rem; +} + +.dropright .dropdown-toggle::after { + display: inline-block; + margin-left: 0.255em; + vertical-align: 0.255em; + content: ""; + border-top: 0.3em solid transparent; + border-right: 0; + border-bottom: 0.3em solid transparent; + border-left: 0.3em solid; +} + +.dropright .dropdown-toggle:empty::after { + margin-left: 0; +} + +.dropright .dropdown-toggle::after { + vertical-align: 0; +} + +.dropleft .dropdown-menu { + top: 0; + right: 100%; + left: auto; + margin-top: 0; + margin-right: 0.125rem; +} + +.dropleft .dropdown-toggle::after { + display: inline-block; + margin-left: 0.255em; + vertical-align: 0.255em; + content: ""; +} + +.dropleft .dropdown-toggle::after { + display: none; +} + +.dropleft .dropdown-toggle::before { + display: inline-block; + margin-right: 0.255em; + vertical-align: 0.255em; + content: ""; + border-top: 0.3em solid transparent; + border-right: 0.3em solid; + border-bottom: 0.3em solid transparent; +} + +.dropleft .dropdown-toggle:empty::after { + margin-left: 0; +} + +.dropleft .dropdown-toggle::before { + vertical-align: 0; +} + +.dropdown-menu[x-placement^="top"], .dropdown-menu[x-placement^="right"], .dropdown-menu[x-placement^="bottom"], .dropdown-menu[x-placement^="left"] { + right: auto; + bottom: auto; +} + +.dropdown-divider { + height: 0; + margin: 0.5rem 0; + overflow: hidden; + border-top: 1px solid #e9ecef; +} + +.dropdown-item { + display: block; + width: 100%; + padding: 0.25rem 1.5rem; + clear: both; + font-weight: 400; + color: #212529; + text-align: inherit; + white-space: nowrap; + background-color: transparent; + border: 0; +} + +.dropdown-item:hover, .dropdown-item:focus { + color: #16181b; + text-decoration: none; + background-color: #f8f9fa; +} + +.dropdown-item.active, .dropdown-item:active { + color: #fff; + text-decoration: none; + background-color: #007bff; +} + +.dropdown-item.disabled, .dropdown-item:disabled { + color: #6c757d; + pointer-events: none; + background-color: transparent; +} + +.dropdown-menu.show { + display: block; +} + +.dropdown-header { + display: block; + padding: 0.5rem 1.5rem; + margin-bottom: 0; + font-size: 0.875rem; + color: #6c757d; + white-space: nowrap; +} + +.dropdown-item-text { + display: block; + padding: 0.25rem 1.5rem; + color: #212529; +} + +.btn-group, +.btn-group-vertical { + position: relative; + display: -ms-inline-flexbox; + display: inline-flex; + vertical-align: middle; +} + +.btn-group > .btn, +.btn-group-vertical > .btn { + position: relative; + -ms-flex: 1 1 auto; + flex: 1 1 auto; +} + +.btn-group > .btn:hover, +.btn-group-vertical > .btn:hover { + z-index: 1; +} + +.btn-group > .btn:focus, .btn-group > .btn:active, .btn-group > .btn.active, +.btn-group-vertical > .btn:focus, +.btn-group-vertical > .btn:active, +.btn-group-vertical > .btn.active { + z-index: 1; +} + +.btn-toolbar { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + -ms-flex-pack: start; + justify-content: flex-start; +} + +.btn-toolbar .input-group { + width: auto; +} + +.btn-group > .btn:not(:first-child), +.btn-group > .btn-group:not(:first-child) { + margin-left: -1px; +} + +.btn-group > .btn:not(:last-child):not(.dropdown-toggle), +.btn-group > .btn-group:not(:last-child) > .btn { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} + +.btn-group > .btn:not(:first-child), +.btn-group > .btn-group:not(:first-child) > .btn { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} + +.dropdown-toggle-split { + padding-right: 0.5625rem; + padding-left: 0.5625rem; +} + +.dropdown-toggle-split::after, +.dropup .dropdown-toggle-split::after, +.dropright .dropdown-toggle-split::after { + margin-left: 0; +} + +.dropleft .dropdown-toggle-split::before { + margin-right: 0; +} + +.btn-sm + .dropdown-toggle-split, .btn-group-sm > .btn + .dropdown-toggle-split { + padding-right: 0.375rem; + padding-left: 0.375rem; +} + +.btn-lg + .dropdown-toggle-split, .btn-group-lg > .btn + .dropdown-toggle-split { + padding-right: 0.75rem; + padding-left: 0.75rem; +} + +.btn-group-vertical { + -ms-flex-direction: column; + flex-direction: column; + -ms-flex-align: start; + align-items: flex-start; + -ms-flex-pack: center; + justify-content: center; +} + +.btn-group-vertical > .btn, +.btn-group-vertical > .btn-group { + width: 100%; +} + +.btn-group-vertical > .btn:not(:first-child), +.btn-group-vertical > .btn-group:not(:first-child) { + margin-top: -1px; +} + +.btn-group-vertical > .btn:not(:last-child):not(.dropdown-toggle), +.btn-group-vertical > .btn-group:not(:last-child) > .btn { + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} + +.btn-group-vertical > .btn:not(:first-child), +.btn-group-vertical > .btn-group:not(:first-child) > .btn { + border-top-left-radius: 0; + border-top-right-radius: 0; +} + +.btn-group-toggle > .btn, +.btn-group-toggle > .btn-group > .btn { + margin-bottom: 0; +} + +.btn-group-toggle > .btn input[type="radio"], +.btn-group-toggle > .btn input[type="checkbox"], +.btn-group-toggle > .btn-group > .btn input[type="radio"], +.btn-group-toggle > .btn-group > .btn input[type="checkbox"] { + position: absolute; + clip: rect(0, 0, 0, 0); + pointer-events: none; +} + +.input-group { + position: relative; + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + -ms-flex-align: stretch; + align-items: stretch; + width: 100%; +} + +.input-group > .form-control, +.input-group > .form-control-plaintext, +.input-group > .custom-select, +.input-group > .custom-file { + position: relative; + -ms-flex: 1 1 0%; + flex: 1 1 0%; + min-width: 0; + margin-bottom: 0; +} + +.input-group > .form-control + .form-control, +.input-group > .form-control + .custom-select, +.input-group > .form-control + .custom-file, +.input-group > .form-control-plaintext + .form-control, +.input-group > .form-control-plaintext + .custom-select, +.input-group > .form-control-plaintext + .custom-file, +.input-group > .custom-select + .form-control, +.input-group > .custom-select + .custom-select, +.input-group > .custom-select + .custom-file, +.input-group > .custom-file + .form-control, +.input-group > .custom-file + .custom-select, +.input-group > .custom-file + .custom-file { + margin-left: -1px; +} + +.input-group > .form-control:focus, +.input-group > .custom-select:focus, +.input-group > .custom-file .custom-file-input:focus ~ .custom-file-label { + z-index: 3; +} + +.input-group > .custom-file .custom-file-input:focus { + z-index: 4; +} + +.input-group > .form-control:not(:last-child), +.input-group > .custom-select:not(:last-child) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} + +.input-group > .form-control:not(:first-child), +.input-group > .custom-select:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} + +.input-group > .custom-file { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; +} + +.input-group > .custom-file:not(:last-child) .custom-file-label, +.input-group > .custom-file:not(:last-child) .custom-file-label::after { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} + +.input-group > .custom-file:not(:first-child) .custom-file-label { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} + +.input-group-prepend, +.input-group-append { + display: -ms-flexbox; + display: flex; +} + +.input-group-prepend .btn, +.input-group-append .btn { + position: relative; + z-index: 2; +} + +.input-group-prepend .btn:focus, +.input-group-append .btn:focus { + z-index: 3; +} + +.input-group-prepend .btn + .btn, +.input-group-prepend .btn + .input-group-text, +.input-group-prepend .input-group-text + .input-group-text, +.input-group-prepend .input-group-text + .btn, +.input-group-append .btn + .btn, +.input-group-append .btn + .input-group-text, +.input-group-append .input-group-text + .input-group-text, +.input-group-append .input-group-text + .btn { + margin-left: -1px; +} + +.input-group-prepend { + margin-right: -1px; +} + +.input-group-append { + margin-left: -1px; +} + +.input-group-text { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + padding: 0.375rem 0.75rem; + margin-bottom: 0; + font-size: 1rem; + font-weight: 400; + line-height: 1.5; + color: #495057; + text-align: center; + white-space: nowrap; + background-color: #e9ecef; + border: 1px solid #ced4da; + border-radius: 0.25rem; +} + +.input-group-text input[type="radio"], +.input-group-text input[type="checkbox"] { + margin-top: 0; +} + +.input-group-lg > .form-control:not(textarea), +.input-group-lg > .custom-select { + height: calc(1.5em + 1rem + 2px); +} + +.input-group-lg > .form-control, +.input-group-lg > .custom-select, +.input-group-lg > .input-group-prepend > .input-group-text, +.input-group-lg > .input-group-append > .input-group-text, +.input-group-lg > .input-group-prepend > .btn, +.input-group-lg > .input-group-append > .btn { + padding: 0.5rem 1rem; + font-size: 1.25rem; + line-height: 1.5; + border-radius: 0.3rem; +} + +.input-group-sm > .form-control:not(textarea), +.input-group-sm > .custom-select { + height: calc(1.5em + 0.5rem + 2px); +} + +.input-group-sm > .form-control, +.input-group-sm > .custom-select, +.input-group-sm > .input-group-prepend > .input-group-text, +.input-group-sm > .input-group-append > .input-group-text, +.input-group-sm > .input-group-prepend > .btn, +.input-group-sm > .input-group-append > .btn { + padding: 0.25rem 0.5rem; + font-size: 0.875rem; + line-height: 1.5; + border-radius: 0.2rem; +} + +.input-group-lg > .custom-select, +.input-group-sm > .custom-select { + padding-right: 1.75rem; +} + +.input-group > .input-group-prepend > .btn, +.input-group > .input-group-prepend > .input-group-text, +.input-group > .input-group-append:not(:last-child) > .btn, +.input-group > .input-group-append:not(:last-child) > .input-group-text, +.input-group > .input-group-append:last-child > .btn:not(:last-child):not(.dropdown-toggle), +.input-group > .input-group-append:last-child > .input-group-text:not(:last-child) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} + +.input-group > .input-group-append > .btn, +.input-group > .input-group-append > .input-group-text, +.input-group > .input-group-prepend:not(:first-child) > .btn, +.input-group > .input-group-prepend:not(:first-child) > .input-group-text, +.input-group > .input-group-prepend:first-child > .btn:not(:first-child), +.input-group > .input-group-prepend:first-child > .input-group-text:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} + +.custom-control { + position: relative; + display: block; + min-height: 1.5rem; + padding-left: 1.5rem; +} + +.custom-control-inline { + display: -ms-inline-flexbox; + display: inline-flex; + margin-right: 1rem; +} + +.custom-control-input { + position: absolute; + left: 0; + z-index: -1; + width: 1rem; + height: 1.25rem; + opacity: 0; +} + +.custom-control-input:checked ~ .custom-control-label::before { + color: #fff; + border-color: #007bff; + background-color: #007bff; +} + +.custom-control-input:focus ~ .custom-control-label::before { + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-control-input:focus:not(:checked) ~ .custom-control-label::before { + border-color: #80bdff; +} + +.custom-control-input:not(:disabled):active ~ .custom-control-label::before { + color: #fff; + background-color: #b3d7ff; + border-color: #b3d7ff; +} + +.custom-control-input[disabled] ~ .custom-control-label, .custom-control-input:disabled ~ .custom-control-label { + color: #6c757d; +} + +.custom-control-input[disabled] ~ .custom-control-label::before, .custom-control-input:disabled ~ .custom-control-label::before { + background-color: #e9ecef; +} + +.custom-control-label { + position: relative; + margin-bottom: 0; + vertical-align: top; +} + +.custom-control-label::before { + position: absolute; + top: 0.25rem; + left: -1.5rem; + display: block; + width: 1rem; + height: 1rem; + pointer-events: none; + content: ""; + background-color: #fff; + border: #adb5bd solid 1px; +} + +.custom-control-label::after { + position: absolute; + top: 0.25rem; + left: -1.5rem; + display: block; + width: 1rem; + height: 1rem; + content: ""; + background: no-repeat 50% / 50% 50%; +} + +.custom-checkbox .custom-control-label::before { + border-radius: 0.25rem; +} + +.custom-checkbox .custom-control-input:checked ~ .custom-control-label::after { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%23fff' d='M6.564.75l-3.59 3.612-1.538-1.55L0 4.26l2.974 2.99L8 2.193z'/%3e%3c/svg%3e"); +} + +.custom-checkbox .custom-control-input:indeterminate ~ .custom-control-label::before { + border-color: #007bff; + background-color: #007bff; +} + +.custom-checkbox .custom-control-input:indeterminate ~ .custom-control-label::after { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='4' viewBox='0 0 4 4'%3e%3cpath stroke='%23fff' d='M0 2h4'/%3e%3c/svg%3e"); +} + +.custom-checkbox .custom-control-input:disabled:checked ~ .custom-control-label::before { + background-color: rgba(0, 123, 255, 0.5); +} + +.custom-checkbox .custom-control-input:disabled:indeterminate ~ .custom-control-label::before { + background-color: rgba(0, 123, 255, 0.5); +} + +.custom-radio .custom-control-label::before { + border-radius: 50%; +} + +.custom-radio .custom-control-input:checked ~ .custom-control-label::after { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' viewBox='-4 -4 8 8'%3e%3ccircle r='3' fill='%23fff'/%3e%3c/svg%3e"); +} + +.custom-radio .custom-control-input:disabled:checked ~ .custom-control-label::before { + background-color: rgba(0, 123, 255, 0.5); +} + +.custom-switch { + padding-left: 2.25rem; +} + +.custom-switch .custom-control-label::before { + left: -2.25rem; + width: 1.75rem; + pointer-events: all; + border-radius: 0.5rem; +} + +.custom-switch .custom-control-label::after { + top: calc(0.25rem + 2px); + left: calc(-2.25rem + 2px); + width: calc(1rem - 4px); + height: calc(1rem - 4px); + background-color: #adb5bd; + border-radius: 0.5rem; + transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out, -webkit-transform 0.15s ease-in-out; + transition: transform 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + transition: transform 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out, -webkit-transform 0.15s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .custom-switch .custom-control-label::after { + transition: none; + } +} + +.custom-switch .custom-control-input:checked ~ .custom-control-label::after { + background-color: #fff; + -webkit-transform: translateX(0.75rem); + transform: translateX(0.75rem); +} + +.custom-switch .custom-control-input:disabled:checked ~ .custom-control-label::before { + background-color: rgba(0, 123, 255, 0.5); +} + +.custom-select { + display: inline-block; + width: 100%; + height: calc(1.5em + 0.75rem + 2px); + padding: 0.375rem 1.75rem 0.375rem 0.75rem; + font-size: 1rem; + font-weight: 400; + line-height: 1.5; + color: #495057; + vertical-align: middle; + background: #fff url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px; + border: 1px solid #ced4da; + border-radius: 0.25rem; + -webkit-appearance: none; + -moz-appearance: none; + appearance: none; +} + +.custom-select:focus { + border-color: #80bdff; + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-select:focus::-ms-value { + color: #495057; + background-color: #fff; +} + +.custom-select[multiple], .custom-select[size]:not([size="1"]) { + height: auto; + padding-right: 0.75rem; + background-image: none; +} + +.custom-select:disabled { + color: #6c757d; + background-color: #e9ecef; +} + +.custom-select::-ms-expand { + display: none; +} + +.custom-select:-moz-focusring { + color: transparent; + text-shadow: 0 0 0 #495057; +} + +.custom-select-sm { + height: calc(1.5em + 0.5rem + 2px); + padding-top: 0.25rem; + padding-bottom: 0.25rem; + padding-left: 0.5rem; + font-size: 0.875rem; +} + +.custom-select-lg { + height: calc(1.5em + 1rem + 2px); + padding-top: 0.5rem; + padding-bottom: 0.5rem; + padding-left: 1rem; + font-size: 1.25rem; +} + +.custom-file { + position: relative; + display: inline-block; + width: 100%; + height: calc(1.5em + 0.75rem + 2px); + margin-bottom: 0; +} + +.custom-file-input { + position: relative; + z-index: 2; + width: 100%; + height: calc(1.5em + 0.75rem + 2px); + margin: 0; + opacity: 0; +} + +.custom-file-input:focus ~ .custom-file-label { + border-color: #80bdff; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-file-input[disabled] ~ .custom-file-label, +.custom-file-input:disabled ~ .custom-file-label { + background-color: #e9ecef; +} + +.custom-file-input:lang(en) ~ .custom-file-label::after { + content: "Browse"; +} + +.custom-file-input ~ .custom-file-label[data-browse]::after { + content: attr(data-browse); +} + +.custom-file-label { + position: absolute; + top: 0; + right: 0; + left: 0; + z-index: 1; + height: calc(1.5em + 0.75rem + 2px); + padding: 0.375rem 0.75rem; + font-weight: 400; + line-height: 1.5; + color: #495057; + background-color: #fff; + border: 1px solid #ced4da; + border-radius: 0.25rem; +} + +.custom-file-label::after { + position: absolute; + top: 0; + right: 0; + bottom: 0; + z-index: 3; + display: block; + height: calc(1.5em + 0.75rem); + padding: 0.375rem 0.75rem; + line-height: 1.5; + color: #495057; + content: "Browse"; + background-color: #e9ecef; + border-left: inherit; + border-radius: 0 0.25rem 0.25rem 0; +} + +.custom-range { + width: 100%; + height: 1.4rem; + padding: 0; + background-color: transparent; + -webkit-appearance: none; + -moz-appearance: none; + appearance: none; +} + +.custom-range:focus { + outline: none; +} + +.custom-range:focus::-webkit-slider-thumb { + box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-range:focus::-moz-range-thumb { + box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-range:focus::-ms-thumb { + box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.custom-range::-moz-focus-outer { + border: 0; +} + +.custom-range::-webkit-slider-thumb { + width: 1rem; + height: 1rem; + margin-top: -0.25rem; + background-color: #007bff; + border: 0; + border-radius: 1rem; + -webkit-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + -webkit-appearance: none; + appearance: none; +} + +@media (prefers-reduced-motion: reduce) { + .custom-range::-webkit-slider-thumb { + -webkit-transition: none; + transition: none; + } +} + +.custom-range::-webkit-slider-thumb:active { + background-color: #b3d7ff; +} + +.custom-range::-webkit-slider-runnable-track { + width: 100%; + height: 0.5rem; + color: transparent; + cursor: pointer; + background-color: #dee2e6; + border-color: transparent; + border-radius: 1rem; +} + +.custom-range::-moz-range-thumb { + width: 1rem; + height: 1rem; + background-color: #007bff; + border: 0; + border-radius: 1rem; + -moz-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + -moz-appearance: none; + appearance: none; +} + +@media (prefers-reduced-motion: reduce) { + .custom-range::-moz-range-thumb { + -moz-transition: none; + transition: none; + } +} + +.custom-range::-moz-range-thumb:active { + background-color: #b3d7ff; +} + +.custom-range::-moz-range-track { + width: 100%; + height: 0.5rem; + color: transparent; + cursor: pointer; + background-color: #dee2e6; + border-color: transparent; + border-radius: 1rem; +} + +.custom-range::-ms-thumb { + width: 1rem; + height: 1rem; + margin-top: 0; + margin-right: 0.2rem; + margin-left: 0.2rem; + background-color: #007bff; + border: 0; + border-radius: 1rem; + -ms-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; + appearance: none; +} + +@media (prefers-reduced-motion: reduce) { + .custom-range::-ms-thumb { + -ms-transition: none; + transition: none; + } +} + +.custom-range::-ms-thumb:active { + background-color: #b3d7ff; +} + +.custom-range::-ms-track { + width: 100%; + height: 0.5rem; + color: transparent; + cursor: pointer; + background-color: transparent; + border-color: transparent; + border-width: 0.5rem; +} + +.custom-range::-ms-fill-lower { + background-color: #dee2e6; + border-radius: 1rem; +} + +.custom-range::-ms-fill-upper { + margin-right: 15px; + background-color: #dee2e6; + border-radius: 1rem; +} + +.custom-range:disabled::-webkit-slider-thumb { + background-color: #adb5bd; +} + +.custom-range:disabled::-webkit-slider-runnable-track { + cursor: default; +} + +.custom-range:disabled::-moz-range-thumb { + background-color: #adb5bd; +} + +.custom-range:disabled::-moz-range-track { + cursor: default; +} + +.custom-range:disabled::-ms-thumb { + background-color: #adb5bd; +} + +.custom-control-label::before, +.custom-file-label, +.custom-select { + transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .custom-control-label::before, + .custom-file-label, + .custom-select { + transition: none; + } +} + +.nav { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + padding-left: 0; + margin-bottom: 0; + list-style: none; +} + +.nav-link { + display: block; + padding: 0.5rem 1rem; +} + +.nav-link:hover, .nav-link:focus { + text-decoration: none; +} + +.nav-link.disabled { + color: #6c757d; + pointer-events: none; + cursor: default; +} + +.nav-tabs { + border-bottom: 1px solid #dee2e6; +} + +.nav-tabs .nav-item { + margin-bottom: -1px; +} + +.nav-tabs .nav-link { + border: 1px solid transparent; + border-top-left-radius: 0.25rem; + border-top-right-radius: 0.25rem; +} + +.nav-tabs .nav-link:hover, .nav-tabs .nav-link:focus { + border-color: #e9ecef #e9ecef #dee2e6; +} + +.nav-tabs .nav-link.disabled { + color: #6c757d; + background-color: transparent; + border-color: transparent; +} + +.nav-tabs .nav-link.active, +.nav-tabs .nav-item.show .nav-link { + color: #495057; + background-color: #fff; + border-color: #dee2e6 #dee2e6 #fff; +} + +.nav-tabs .dropdown-menu { + margin-top: -1px; + border-top-left-radius: 0; + border-top-right-radius: 0; +} + +.nav-pills .nav-link { + border-radius: 0.25rem; +} + +.nav-pills .nav-link.active, +.nav-pills .show > .nav-link { + color: #fff; + background-color: #007bff; +} + +.nav-fill .nav-item { + -ms-flex: 1 1 auto; + flex: 1 1 auto; + text-align: center; +} + +.nav-justified .nav-item { + -ms-flex-preferred-size: 0; + flex-basis: 0; + -ms-flex-positive: 1; + flex-grow: 1; + text-align: center; +} + +.tab-content > .tab-pane { + display: none; +} + +.tab-content > .active { + display: block; +} + +.navbar { + position: relative; + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: justify; + justify-content: space-between; + padding: 0.5rem 1rem; +} + +.navbar .container, +.navbar .container-fluid, .navbar .container-sm, .navbar .container-md, .navbar .container-lg, .navbar .container-xl { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: justify; + justify-content: space-between; +} + +.navbar-brand { + display: inline-block; + padding-top: 0.3125rem; + padding-bottom: 0.3125rem; + margin-right: 1rem; + font-size: 1.25rem; + line-height: inherit; + white-space: nowrap; +} + +.navbar-brand:hover, .navbar-brand:focus { + text-decoration: none; +} + +.navbar-nav { + display: -ms-flexbox; + display: flex; + -ms-flex-direction: column; + flex-direction: column; + padding-left: 0; + margin-bottom: 0; + list-style: none; +} + +.navbar-nav .nav-link { + padding-right: 0; + padding-left: 0; +} + +.navbar-nav .dropdown-menu { + position: static; + float: none; +} + +.navbar-text { + display: inline-block; + padding-top: 0.5rem; + padding-bottom: 0.5rem; +} + +.navbar-collapse { + -ms-flex-preferred-size: 100%; + flex-basis: 100%; + -ms-flex-positive: 1; + flex-grow: 1; + -ms-flex-align: center; + align-items: center; +} + +.navbar-toggler { + padding: 0.25rem 0.75rem; + font-size: 1.25rem; + line-height: 1; + background-color: transparent; + border: 1px solid transparent; + border-radius: 0.25rem; +} + +.navbar-toggler:hover, .navbar-toggler:focus { + text-decoration: none; +} + +.navbar-toggler-icon { + display: inline-block; + width: 1.5em; + height: 1.5em; + vertical-align: middle; + content: ""; + background: no-repeat center center; + background-size: 100% 100%; +} + +@media (max-width: 575.98px) { + .navbar-expand-sm > .container, + .navbar-expand-sm > .container-fluid, .navbar-expand-sm > .container-sm, .navbar-expand-sm > .container-md, .navbar-expand-sm > .container-lg, .navbar-expand-sm > .container-xl { + padding-right: 0; + padding-left: 0; + } +} + +@media (min-width: 576px) { + .navbar-expand-sm { + -ms-flex-flow: row nowrap; + flex-flow: row nowrap; + -ms-flex-pack: start; + justify-content: flex-start; + } + .navbar-expand-sm .navbar-nav { + -ms-flex-direction: row; + flex-direction: row; + } + .navbar-expand-sm .navbar-nav .dropdown-menu { + position: absolute; + } + .navbar-expand-sm .navbar-nav .nav-link { + padding-right: 0.5rem; + padding-left: 0.5rem; + } + .navbar-expand-sm > .container, + .navbar-expand-sm > .container-fluid, .navbar-expand-sm > .container-sm, .navbar-expand-sm > .container-md, .navbar-expand-sm > .container-lg, .navbar-expand-sm > .container-xl { + -ms-flex-wrap: nowrap; + flex-wrap: nowrap; + } + .navbar-expand-sm .navbar-collapse { + display: -ms-flexbox !important; + display: flex !important; + -ms-flex-preferred-size: auto; + flex-basis: auto; + } + .navbar-expand-sm .navbar-toggler { + display: none; + } +} + +@media (max-width: 767.98px) { + .navbar-expand-md > .container, + .navbar-expand-md > .container-fluid, .navbar-expand-md > .container-sm, .navbar-expand-md > .container-md, .navbar-expand-md > .container-lg, .navbar-expand-md > .container-xl { + padding-right: 0; + padding-left: 0; + } +} + +@media (min-width: 768px) { + .navbar-expand-md { + -ms-flex-flow: row nowrap; + flex-flow: row nowrap; + -ms-flex-pack: start; + justify-content: flex-start; + } + .navbar-expand-md .navbar-nav { + -ms-flex-direction: row; + flex-direction: row; + } + .navbar-expand-md .navbar-nav .dropdown-menu { + position: absolute; + } + .navbar-expand-md .navbar-nav .nav-link { + padding-right: 0.5rem; + padding-left: 0.5rem; + } + .navbar-expand-md > .container, + .navbar-expand-md > .container-fluid, .navbar-expand-md > .container-sm, .navbar-expand-md > .container-md, .navbar-expand-md > .container-lg, .navbar-expand-md > .container-xl { + -ms-flex-wrap: nowrap; + flex-wrap: nowrap; + } + .navbar-expand-md .navbar-collapse { + display: -ms-flexbox !important; + display: flex !important; + -ms-flex-preferred-size: auto; + flex-basis: auto; + } + .navbar-expand-md .navbar-toggler { + display: none; + } +} + +@media (max-width: 991.98px) { + .navbar-expand-lg > .container, + .navbar-expand-lg > .container-fluid, .navbar-expand-lg > .container-sm, .navbar-expand-lg > .container-md, .navbar-expand-lg > .container-lg, .navbar-expand-lg > .container-xl { + padding-right: 0; + padding-left: 0; + } +} + +@media (min-width: 992px) { + .navbar-expand-lg { + -ms-flex-flow: row nowrap; + flex-flow: row nowrap; + -ms-flex-pack: start; + justify-content: flex-start; + } + .navbar-expand-lg .navbar-nav { + -ms-flex-direction: row; + flex-direction: row; + } + .navbar-expand-lg .navbar-nav .dropdown-menu { + position: absolute; + } + .navbar-expand-lg .navbar-nav .nav-link { + padding-right: 0.5rem; + padding-left: 0.5rem; + } + .navbar-expand-lg > .container, + .navbar-expand-lg > .container-fluid, .navbar-expand-lg > .container-sm, .navbar-expand-lg > .container-md, .navbar-expand-lg > .container-lg, .navbar-expand-lg > .container-xl { + -ms-flex-wrap: nowrap; + flex-wrap: nowrap; + } + .navbar-expand-lg .navbar-collapse { + display: -ms-flexbox !important; + display: flex !important; + -ms-flex-preferred-size: auto; + flex-basis: auto; + } + .navbar-expand-lg .navbar-toggler { + display: none; + } +} + +@media (max-width: 1199.98px) { + .navbar-expand-xl > .container, + .navbar-expand-xl > .container-fluid, .navbar-expand-xl > .container-sm, .navbar-expand-xl > .container-md, .navbar-expand-xl > .container-lg, .navbar-expand-xl > .container-xl { + padding-right: 0; + padding-left: 0; + } +} + +@media (min-width: 1200px) { + .navbar-expand-xl { + -ms-flex-flow: row nowrap; + flex-flow: row nowrap; + -ms-flex-pack: start; + justify-content: flex-start; + } + .navbar-expand-xl .navbar-nav { + -ms-flex-direction: row; + flex-direction: row; + } + .navbar-expand-xl .navbar-nav .dropdown-menu { + position: absolute; + } + .navbar-expand-xl .navbar-nav .nav-link { + padding-right: 0.5rem; + padding-left: 0.5rem; + } + .navbar-expand-xl > .container, + .navbar-expand-xl > .container-fluid, .navbar-expand-xl > .container-sm, .navbar-expand-xl > .container-md, .navbar-expand-xl > .container-lg, .navbar-expand-xl > .container-xl { + -ms-flex-wrap: nowrap; + flex-wrap: nowrap; + } + .navbar-expand-xl .navbar-collapse { + display: -ms-flexbox !important; + display: flex !important; + -ms-flex-preferred-size: auto; + flex-basis: auto; + } + .navbar-expand-xl .navbar-toggler { + display: none; + } +} + +.navbar-expand { + -ms-flex-flow: row nowrap; + flex-flow: row nowrap; + -ms-flex-pack: start; + justify-content: flex-start; +} + +.navbar-expand > .container, +.navbar-expand > .container-fluid, .navbar-expand > .container-sm, .navbar-expand > .container-md, .navbar-expand > .container-lg, .navbar-expand > .container-xl { + padding-right: 0; + padding-left: 0; +} + +.navbar-expand .navbar-nav { + -ms-flex-direction: row; + flex-direction: row; +} + +.navbar-expand .navbar-nav .dropdown-menu { + position: absolute; +} + +.navbar-expand .navbar-nav .nav-link { + padding-right: 0.5rem; + padding-left: 0.5rem; +} + +.navbar-expand > .container, +.navbar-expand > .container-fluid, .navbar-expand > .container-sm, .navbar-expand > .container-md, .navbar-expand > .container-lg, .navbar-expand > .container-xl { + -ms-flex-wrap: nowrap; + flex-wrap: nowrap; +} + +.navbar-expand .navbar-collapse { + display: -ms-flexbox !important; + display: flex !important; + -ms-flex-preferred-size: auto; + flex-basis: auto; +} + +.navbar-expand .navbar-toggler { + display: none; +} + +.navbar-light .navbar-brand { + color: rgba(0, 0, 0, 0.9); +} + +.navbar-light .navbar-brand:hover, .navbar-light .navbar-brand:focus { + color: rgba(0, 0, 0, 0.9); +} + +.navbar-light .navbar-nav .nav-link { + color: rgba(0, 0, 0, 0.5); +} + +.navbar-light .navbar-nav .nav-link:hover, .navbar-light .navbar-nav .nav-link:focus { + color: rgba(0, 0, 0, 0.7); +} + +.navbar-light .navbar-nav .nav-link.disabled { + color: rgba(0, 0, 0, 0.3); +} + +.navbar-light .navbar-nav .show > .nav-link, +.navbar-light .navbar-nav .active > .nav-link, +.navbar-light .navbar-nav .nav-link.show, +.navbar-light .navbar-nav .nav-link.active { + color: rgba(0, 0, 0, 0.9); +} + +.navbar-light .navbar-toggler { + color: rgba(0, 0, 0, 0.5); + border-color: rgba(0, 0, 0, 0.1); +} + +.navbar-light .navbar-toggler-icon { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30' viewBox='0 0 30 30'%3e%3cpath stroke='rgba(0, 0, 0, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3e%3c/svg%3e"); +} + +.navbar-light .navbar-text { + color: rgba(0, 0, 0, 0.5); +} + +.navbar-light .navbar-text a { + color: rgba(0, 0, 0, 0.9); +} + +.navbar-light .navbar-text a:hover, .navbar-light .navbar-text a:focus { + color: rgba(0, 0, 0, 0.9); +} + +.navbar-dark .navbar-brand { + color: #fff; +} + +.navbar-dark .navbar-brand:hover, .navbar-dark .navbar-brand:focus { + color: #fff; +} + +.navbar-dark .navbar-nav .nav-link { + color: rgba(255, 255, 255, 0.5); +} + +.navbar-dark .navbar-nav .nav-link:hover, .navbar-dark .navbar-nav .nav-link:focus { + color: rgba(255, 255, 255, 0.75); +} + +.navbar-dark .navbar-nav .nav-link.disabled { + color: rgba(255, 255, 255, 0.25); +} + +.navbar-dark .navbar-nav .show > .nav-link, +.navbar-dark .navbar-nav .active > .nav-link, +.navbar-dark .navbar-nav .nav-link.show, +.navbar-dark .navbar-nav .nav-link.active { + color: #fff; +} + +.navbar-dark .navbar-toggler { + color: rgba(255, 255, 255, 0.5); + border-color: rgba(255, 255, 255, 0.1); +} + +.navbar-dark .navbar-toggler-icon { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30' viewBox='0 0 30 30'%3e%3cpath stroke='rgba(255, 255, 255, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3e%3c/svg%3e"); +} + +.navbar-dark .navbar-text { + color: rgba(255, 255, 255, 0.5); +} + +.navbar-dark .navbar-text a { + color: #fff; +} + +.navbar-dark .navbar-text a:hover, .navbar-dark .navbar-text a:focus { + color: #fff; +} + +.card { + position: relative; + display: -ms-flexbox; + display: flex; + -ms-flex-direction: column; + flex-direction: column; + min-width: 0; + word-wrap: break-word; + background-color: #fff; + background-clip: border-box; + border: 1px solid rgba(0, 0, 0, 0.125); + border-radius: 0.25rem; +} + +.card > hr { + margin-right: 0; + margin-left: 0; +} + +.card > .list-group:first-child .list-group-item:first-child { + border-top-left-radius: 0.25rem; + border-top-right-radius: 0.25rem; +} + +.card > .list-group:last-child .list-group-item:last-child { + border-bottom-right-radius: 0.25rem; + border-bottom-left-radius: 0.25rem; +} + +.card-body { + -ms-flex: 1 1 auto; + flex: 1 1 auto; + min-height: 1px; + padding: 1.25rem; +} + +.card-title { + margin-bottom: 0.75rem; +} + +.card-subtitle { + margin-top: -0.375rem; + margin-bottom: 0; +} + +.card-text:last-child { + margin-bottom: 0; +} + +.card-link:hover { + text-decoration: none; +} + +.card-link + .card-link { + margin-left: 1.25rem; +} + +.card-header { + padding: 0.75rem 1.25rem; + margin-bottom: 0; + background-color: rgba(0, 0, 0, 0.03); + border-bottom: 1px solid rgba(0, 0, 0, 0.125); +} + +.card-header:first-child { + border-radius: calc(0.25rem - 1px) calc(0.25rem - 1px) 0 0; +} + +.card-header + .list-group .list-group-item:first-child { + border-top: 0; +} + +.card-footer { + padding: 0.75rem 1.25rem; + background-color: rgba(0, 0, 0, 0.03); + border-top: 1px solid rgba(0, 0, 0, 0.125); +} + +.card-footer:last-child { + border-radius: 0 0 calc(0.25rem - 1px) calc(0.25rem - 1px); +} + +.card-header-tabs { + margin-right: -0.625rem; + margin-bottom: -0.75rem; + margin-left: -0.625rem; + border-bottom: 0; +} + +.card-header-pills { + margin-right: -0.625rem; + margin-left: -0.625rem; +} + +.card-img-overlay { + position: absolute; + top: 0; + right: 0; + bottom: 0; + left: 0; + padding: 1.25rem; +} + +.card-img, +.card-img-top, +.card-img-bottom { + -ms-flex-negative: 0; + flex-shrink: 0; + width: 100%; +} + +.card-img, +.card-img-top { + border-top-left-radius: calc(0.25rem - 1px); + border-top-right-radius: calc(0.25rem - 1px); +} + +.card-img, +.card-img-bottom { + border-bottom-right-radius: calc(0.25rem - 1px); + border-bottom-left-radius: calc(0.25rem - 1px); +} + +.card-deck .card { + margin-bottom: 15px; +} + +@media (min-width: 576px) { + .card-deck { + display: -ms-flexbox; + display: flex; + -ms-flex-flow: row wrap; + flex-flow: row wrap; + margin-right: -15px; + margin-left: -15px; + } + .card-deck .card { + -ms-flex: 1 0 0%; + flex: 1 0 0%; + margin-right: 15px; + margin-bottom: 0; + margin-left: 15px; + } +} + +.card-group > .card { + margin-bottom: 15px; +} + +@media (min-width: 576px) { + .card-group { + display: -ms-flexbox; + display: flex; + -ms-flex-flow: row wrap; + flex-flow: row wrap; + } + .card-group > .card { + -ms-flex: 1 0 0%; + flex: 1 0 0%; + margin-bottom: 0; + } + .card-group > .card + .card { + margin-left: 0; + border-left: 0; + } + .card-group > .card:not(:last-child) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; + } + .card-group > .card:not(:last-child) .card-img-top, + .card-group > .card:not(:last-child) .card-header { + border-top-right-radius: 0; + } + .card-group > .card:not(:last-child) .card-img-bottom, + .card-group > .card:not(:last-child) .card-footer { + border-bottom-right-radius: 0; + } + .card-group > .card:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; + } + .card-group > .card:not(:first-child) .card-img-top, + .card-group > .card:not(:first-child) .card-header { + border-top-left-radius: 0; + } + .card-group > .card:not(:first-child) .card-img-bottom, + .card-group > .card:not(:first-child) .card-footer { + border-bottom-left-radius: 0; + } +} + +.card-columns .card { + margin-bottom: 0.75rem; +} + +@media (min-width: 576px) { + .card-columns { + -webkit-column-count: 3; + -moz-column-count: 3; + column-count: 3; + -webkit-column-gap: 1.25rem; + -moz-column-gap: 1.25rem; + column-gap: 1.25rem; + orphans: 1; + widows: 1; + } + .card-columns .card { + display: inline-block; + width: 100%; + } +} + +.accordion > .card { + overflow: hidden; +} + +.accordion > .card:not(:last-of-type) { + border-bottom: 0; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} + +.accordion > .card:not(:first-of-type) { + border-top-left-radius: 0; + border-top-right-radius: 0; +} + +.accordion > .card > .card-header { + border-radius: 0; + margin-bottom: -1px; +} + +.breadcrumb { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + padding: 0.75rem 1rem; + margin-bottom: 1rem; + list-style: none; + background-color: #e9ecef; + border-radius: 0.25rem; +} + +.breadcrumb-item + .breadcrumb-item { + padding-left: 0.5rem; +} + +.breadcrumb-item + .breadcrumb-item::before { + display: inline-block; + padding-right: 0.5rem; + color: #6c757d; + content: "/"; +} + +.breadcrumb-item + .breadcrumb-item:hover::before { + text-decoration: underline; +} + +.breadcrumb-item + .breadcrumb-item:hover::before { + text-decoration: none; +} + +.breadcrumb-item.active { + color: #6c757d; +} + +.pagination { + display: -ms-flexbox; + display: flex; + padding-left: 0; + list-style: none; + border-radius: 0.25rem; +} + +.page-link { + position: relative; + display: block; + padding: 0.5rem 0.75rem; + margin-left: -1px; + line-height: 1.25; + color: #007bff; + background-color: #fff; + border: 1px solid #dee2e6; +} + +.page-link:hover { + z-index: 2; + color: #0056b3; + text-decoration: none; + background-color: #e9ecef; + border-color: #dee2e6; +} + +.page-link:focus { + z-index: 3; + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.25); +} + +.page-item:first-child .page-link { + margin-left: 0; + border-top-left-radius: 0.25rem; + border-bottom-left-radius: 0.25rem; +} + +.page-item:last-child .page-link { + border-top-right-radius: 0.25rem; + border-bottom-right-radius: 0.25rem; +} + +.page-item.active .page-link { + z-index: 3; + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.page-item.disabled .page-link { + color: #6c757d; + pointer-events: none; + cursor: auto; + background-color: #fff; + border-color: #dee2e6; +} + +.pagination-lg .page-link { + padding: 0.75rem 1.5rem; + font-size: 1.25rem; + line-height: 1.5; +} + +.pagination-lg .page-item:first-child .page-link { + border-top-left-radius: 0.3rem; + border-bottom-left-radius: 0.3rem; +} + +.pagination-lg .page-item:last-child .page-link { + border-top-right-radius: 0.3rem; + border-bottom-right-radius: 0.3rem; +} + +.pagination-sm .page-link { + padding: 0.25rem 0.5rem; + font-size: 0.875rem; + line-height: 1.5; +} + +.pagination-sm .page-item:first-child .page-link { + border-top-left-radius: 0.2rem; + border-bottom-left-radius: 0.2rem; +} + +.pagination-sm .page-item:last-child .page-link { + border-top-right-radius: 0.2rem; + border-bottom-right-radius: 0.2rem; +} + +.badge { + display: inline-block; + padding: 0.25em 0.4em; + font-size: 75%; + font-weight: 700; + line-height: 1; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + border-radius: 0.25rem; + transition: color 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .badge { + transition: none; + } +} + +a.badge:hover, a.badge:focus { + text-decoration: none; +} + +.badge:empty { + display: none; +} + +.btn .badge { + position: relative; + top: -1px; +} + +.badge-pill { + padding-right: 0.6em; + padding-left: 0.6em; + border-radius: 10rem; +} + +.badge-primary { + color: #fff; + background-color: #007bff; +} + +a.badge-primary:hover, a.badge-primary:focus { + color: #fff; + background-color: #0062cc; +} + +a.badge-primary:focus, a.badge-primary.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(0, 123, 255, 0.5); +} + +.badge-secondary { + color: #fff; + background-color: #6c757d; +} + +a.badge-secondary:hover, a.badge-secondary:focus { + color: #fff; + background-color: #545b62; +} + +a.badge-secondary:focus, a.badge-secondary.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(108, 117, 125, 0.5); +} + +.badge-success { + color: #fff; + background-color: #28a745; +} + +a.badge-success:hover, a.badge-success:focus { + color: #fff; + background-color: #1e7e34; +} + +a.badge-success:focus, a.badge-success.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.5); +} + +.badge-info { + color: #fff; + background-color: #17a2b8; +} + +a.badge-info:hover, a.badge-info:focus { + color: #fff; + background-color: #117a8b; +} + +a.badge-info:focus, a.badge-info.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(23, 162, 184, 0.5); +} + +.badge-warning { + color: #212529; + background-color: #ffc107; +} + +a.badge-warning:hover, a.badge-warning:focus { + color: #212529; + background-color: #d39e00; +} + +a.badge-warning:focus, a.badge-warning.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(255, 193, 7, 0.5); +} + +.badge-danger { + color: #fff; + background-color: #dc3545; +} + +a.badge-danger:hover, a.badge-danger:focus { + color: #fff; + background-color: #bd2130; +} + +a.badge-danger:focus, a.badge-danger.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.5); +} + +.badge-light { + color: #212529; + background-color: #f8f9fa; +} + +a.badge-light:hover, a.badge-light:focus { + color: #212529; + background-color: #dae0e5; +} + +a.badge-light:focus, a.badge-light.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(248, 249, 250, 0.5); +} + +.badge-dark { + color: #fff; + background-color: #343a40; +} + +a.badge-dark:hover, a.badge-dark:focus { + color: #fff; + background-color: #1d2124; +} + +a.badge-dark:focus, a.badge-dark.focus { + outline: 0; + box-shadow: 0 0 0 0.2rem rgba(52, 58, 64, 0.5); +} + +.jumbotron { + padding: 2rem 1rem; + margin-bottom: 2rem; + background-color: #e9ecef; + border-radius: 0.3rem; +} + +@media (min-width: 576px) { + .jumbotron { + padding: 4rem 2rem; + } +} + +.jumbotron-fluid { + padding-right: 0; + padding-left: 0; + border-radius: 0; +} + +.alert { + position: relative; + padding: 0.75rem 1.25rem; + margin-bottom: 1rem; + border: 1px solid transparent; + border-radius: 0.25rem; +} + +.alert-heading { + color: inherit; +} + +.alert-link { + font-weight: 700; +} + +.alert-dismissible { + padding-right: 4rem; +} + +.alert-dismissible .close { + position: absolute; + top: 0; + right: 0; + padding: 0.75rem 1.25rem; + color: inherit; +} + +.alert-primary { + color: #004085; + background-color: #cce5ff; + border-color: #b8daff; +} + +.alert-primary hr { + border-top-color: #9fcdff; +} + +.alert-primary .alert-link { + color: #002752; +} + +.alert-secondary { + color: #383d41; + background-color: #e2e3e5; + border-color: #d6d8db; +} + +.alert-secondary hr { + border-top-color: #c8cbcf; +} + +.alert-secondary .alert-link { + color: #202326; +} + +.alert-success { + color: #155724; + background-color: #d4edda; + border-color: #c3e6cb; +} + +.alert-success hr { + border-top-color: #b1dfbb; +} + +.alert-success .alert-link { + color: #0b2e13; +} + +.alert-info { + color: #0c5460; + background-color: #d1ecf1; + border-color: #bee5eb; +} + +.alert-info hr { + border-top-color: #abdde5; +} + +.alert-info .alert-link { + color: #062c33; +} + +.alert-warning { + color: #856404; + background-color: #fff3cd; + border-color: #ffeeba; +} + +.alert-warning hr { + border-top-color: #ffe8a1; +} + +.alert-warning .alert-link { + color: #533f03; +} + +.alert-danger { + color: #721c24; + background-color: #f8d7da; + border-color: #f5c6cb; +} + +.alert-danger hr { + border-top-color: #f1b0b7; +} + +.alert-danger .alert-link { + color: #491217; +} + +.alert-light { + color: #818182; + background-color: #fefefe; + border-color: #fdfdfe; +} + +.alert-light hr { + border-top-color: #ececf6; +} + +.alert-light .alert-link { + color: #686868; +} + +.alert-dark { + color: #1b1e21; + background-color: #d6d8d9; + border-color: #c6c8ca; +} + +.alert-dark hr { + border-top-color: #b9bbbe; +} + +.alert-dark .alert-link { + color: #040505; +} + +@-webkit-keyframes progress-bar-stripes { + from { + background-position: 1rem 0; + } + to { + background-position: 0 0; + } +} + +@keyframes progress-bar-stripes { + from { + background-position: 1rem 0; + } + to { + background-position: 0 0; + } +} + +.progress { + display: -ms-flexbox; + display: flex; + height: 1rem; + overflow: hidden; + font-size: 0.75rem; + background-color: #e9ecef; + border-radius: 0.25rem; +} + +.progress-bar { + display: -ms-flexbox; + display: flex; + -ms-flex-direction: column; + flex-direction: column; + -ms-flex-pack: center; + justify-content: center; + overflow: hidden; + color: #fff; + text-align: center; + white-space: nowrap; + background-color: #007bff; + transition: width 0.6s ease; +} + +@media (prefers-reduced-motion: reduce) { + .progress-bar { + transition: none; + } +} + +.progress-bar-striped { + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-size: 1rem 1rem; +} + +.progress-bar-animated { + -webkit-animation: progress-bar-stripes 1s linear infinite; + animation: progress-bar-stripes 1s linear infinite; +} + +@media (prefers-reduced-motion: reduce) { + .progress-bar-animated { + -webkit-animation: none; + animation: none; + } +} + +.media { + display: -ms-flexbox; + display: flex; + -ms-flex-align: start; + align-items: flex-start; +} + +.media-body { + -ms-flex: 1; + flex: 1; +} + +.list-group { + display: -ms-flexbox; + display: flex; + -ms-flex-direction: column; + flex-direction: column; + padding-left: 0; + margin-bottom: 0; +} + +.list-group-item-action { + width: 100%; + color: #495057; + text-align: inherit; +} + +.list-group-item-action:hover, .list-group-item-action:focus { + z-index: 1; + color: #495057; + text-decoration: none; + background-color: #f8f9fa; +} + +.list-group-item-action:active { + color: #212529; + background-color: #e9ecef; +} + +.list-group-item { + position: relative; + display: block; + padding: 0.75rem 1.25rem; + background-color: #fff; + border: 1px solid rgba(0, 0, 0, 0.125); +} + +.list-group-item:first-child { + border-top-left-radius: 0.25rem; + border-top-right-radius: 0.25rem; +} + +.list-group-item:last-child { + border-bottom-right-radius: 0.25rem; + border-bottom-left-radius: 0.25rem; +} + +.list-group-item.disabled, .list-group-item:disabled { + color: #6c757d; + pointer-events: none; + background-color: #fff; +} + +.list-group-item.active { + z-index: 2; + color: #fff; + background-color: #007bff; + border-color: #007bff; +} + +.list-group-item + .list-group-item { + border-top-width: 0; +} + +.list-group-item + .list-group-item.active { + margin-top: -1px; + border-top-width: 1px; +} + +.list-group-horizontal { + -ms-flex-direction: row; + flex-direction: row; +} + +.list-group-horizontal .list-group-item:first-child { + border-bottom-left-radius: 0.25rem; + border-top-right-radius: 0; +} + +.list-group-horizontal .list-group-item:last-child { + border-top-right-radius: 0.25rem; + border-bottom-left-radius: 0; +} + +.list-group-horizontal .list-group-item.active { + margin-top: 0; +} + +.list-group-horizontal .list-group-item + .list-group-item { + border-top-width: 1px; + border-left-width: 0; +} + +.list-group-horizontal .list-group-item + .list-group-item.active { + margin-left: -1px; + border-left-width: 1px; +} + +@media (min-width: 576px) { + .list-group-horizontal-sm { + -ms-flex-direction: row; + flex-direction: row; + } + .list-group-horizontal-sm .list-group-item:first-child { + border-bottom-left-radius: 0.25rem; + border-top-right-radius: 0; + } + .list-group-horizontal-sm .list-group-item:last-child { + border-top-right-radius: 0.25rem; + border-bottom-left-radius: 0; + } + .list-group-horizontal-sm .list-group-item.active { + margin-top: 0; + } + .list-group-horizontal-sm .list-group-item + .list-group-item { + border-top-width: 1px; + border-left-width: 0; + } + .list-group-horizontal-sm .list-group-item + .list-group-item.active { + margin-left: -1px; + border-left-width: 1px; + } +} + +@media (min-width: 768px) { + .list-group-horizontal-md { + -ms-flex-direction: row; + flex-direction: row; + } + .list-group-horizontal-md .list-group-item:first-child { + border-bottom-left-radius: 0.25rem; + border-top-right-radius: 0; + } + .list-group-horizontal-md .list-group-item:last-child { + border-top-right-radius: 0.25rem; + border-bottom-left-radius: 0; + } + .list-group-horizontal-md .list-group-item.active { + margin-top: 0; + } + .list-group-horizontal-md .list-group-item + .list-group-item { + border-top-width: 1px; + border-left-width: 0; + } + .list-group-horizontal-md .list-group-item + .list-group-item.active { + margin-left: -1px; + border-left-width: 1px; + } +} + +@media (min-width: 992px) { + .list-group-horizontal-lg { + -ms-flex-direction: row; + flex-direction: row; + } + .list-group-horizontal-lg .list-group-item:first-child { + border-bottom-left-radius: 0.25rem; + border-top-right-radius: 0; + } + .list-group-horizontal-lg .list-group-item:last-child { + border-top-right-radius: 0.25rem; + border-bottom-left-radius: 0; + } + .list-group-horizontal-lg .list-group-item.active { + margin-top: 0; + } + .list-group-horizontal-lg .list-group-item + .list-group-item { + border-top-width: 1px; + border-left-width: 0; + } + .list-group-horizontal-lg .list-group-item + .list-group-item.active { + margin-left: -1px; + border-left-width: 1px; + } +} + +@media (min-width: 1200px) { + .list-group-horizontal-xl { + -ms-flex-direction: row; + flex-direction: row; + } + .list-group-horizontal-xl .list-group-item:first-child { + border-bottom-left-radius: 0.25rem; + border-top-right-radius: 0; + } + .list-group-horizontal-xl .list-group-item:last-child { + border-top-right-radius: 0.25rem; + border-bottom-left-radius: 0; + } + .list-group-horizontal-xl .list-group-item.active { + margin-top: 0; + } + .list-group-horizontal-xl .list-group-item + .list-group-item { + border-top-width: 1px; + border-left-width: 0; + } + .list-group-horizontal-xl .list-group-item + .list-group-item.active { + margin-left: -1px; + border-left-width: 1px; + } +} + +.list-group-flush .list-group-item { + border-right-width: 0; + border-left-width: 0; + border-radius: 0; +} + +.list-group-flush .list-group-item:first-child { + border-top-width: 0; +} + +.list-group-flush:last-child .list-group-item:last-child { + border-bottom-width: 0; +} + +.list-group-item-primary { + color: #004085; + background-color: #b8daff; +} + +.list-group-item-primary.list-group-item-action:hover, .list-group-item-primary.list-group-item-action:focus { + color: #004085; + background-color: #9fcdff; +} + +.list-group-item-primary.list-group-item-action.active { + color: #fff; + background-color: #004085; + border-color: #004085; +} + +.list-group-item-secondary { + color: #383d41; + background-color: #d6d8db; +} + +.list-group-item-secondary.list-group-item-action:hover, .list-group-item-secondary.list-group-item-action:focus { + color: #383d41; + background-color: #c8cbcf; +} + +.list-group-item-secondary.list-group-item-action.active { + color: #fff; + background-color: #383d41; + border-color: #383d41; +} + +.list-group-item-success { + color: #155724; + background-color: #c3e6cb; +} + +.list-group-item-success.list-group-item-action:hover, .list-group-item-success.list-group-item-action:focus { + color: #155724; + background-color: #b1dfbb; +} + +.list-group-item-success.list-group-item-action.active { + color: #fff; + background-color: #155724; + border-color: #155724; +} + +.list-group-item-info { + color: #0c5460; + background-color: #bee5eb; +} + +.list-group-item-info.list-group-item-action:hover, .list-group-item-info.list-group-item-action:focus { + color: #0c5460; + background-color: #abdde5; +} + +.list-group-item-info.list-group-item-action.active { + color: #fff; + background-color: #0c5460; + border-color: #0c5460; +} + +.list-group-item-warning { + color: #856404; + background-color: #ffeeba; +} + +.list-group-item-warning.list-group-item-action:hover, .list-group-item-warning.list-group-item-action:focus { + color: #856404; + background-color: #ffe8a1; +} + +.list-group-item-warning.list-group-item-action.active { + color: #fff; + background-color: #856404; + border-color: #856404; +} + +.list-group-item-danger { + color: #721c24; + background-color: #f5c6cb; +} + +.list-group-item-danger.list-group-item-action:hover, .list-group-item-danger.list-group-item-action:focus { + color: #721c24; + background-color: #f1b0b7; +} + +.list-group-item-danger.list-group-item-action.active { + color: #fff; + background-color: #721c24; + border-color: #721c24; +} + +.list-group-item-light { + color: #818182; + background-color: #fdfdfe; +} + +.list-group-item-light.list-group-item-action:hover, .list-group-item-light.list-group-item-action:focus { + color: #818182; + background-color: #ececf6; +} + +.list-group-item-light.list-group-item-action.active { + color: #fff; + background-color: #818182; + border-color: #818182; +} + +.list-group-item-dark { + color: #1b1e21; + background-color: #c6c8ca; +} + +.list-group-item-dark.list-group-item-action:hover, .list-group-item-dark.list-group-item-action:focus { + color: #1b1e21; + background-color: #b9bbbe; +} + +.list-group-item-dark.list-group-item-action.active { + color: #fff; + background-color: #1b1e21; + border-color: #1b1e21; +} + +.close { + float: right; + font-size: 1.5rem; + font-weight: 700; + line-height: 1; + color: #000; + text-shadow: 0 1px 0 #fff; + opacity: .5; +} + +.close:hover { + color: #000; + text-decoration: none; +} + +.close:not(:disabled):not(.disabled):hover, .close:not(:disabled):not(.disabled):focus { + opacity: .75; +} + +button.close { + padding: 0; + background-color: transparent; + border: 0; + -webkit-appearance: none; + -moz-appearance: none; + appearance: none; +} + +a.close.disabled { + pointer-events: none; +} + +.toast { + max-width: 350px; + overflow: hidden; + font-size: 0.875rem; + background-color: rgba(255, 255, 255, 0.85); + background-clip: padding-box; + border: 1px solid rgba(0, 0, 0, 0.1); + box-shadow: 0 0.25rem 0.75rem rgba(0, 0, 0, 0.1); + -webkit-backdrop-filter: blur(10px); + backdrop-filter: blur(10px); + opacity: 0; + border-radius: 0.25rem; +} + +.toast:not(:last-child) { + margin-bottom: 0.75rem; +} + +.toast.showing { + opacity: 1; +} + +.toast.show { + display: block; + opacity: 1; +} + +.toast.hide { + display: none; +} + +.toast-header { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + padding: 0.25rem 0.75rem; + color: #6c757d; + background-color: rgba(255, 255, 255, 0.85); + background-clip: padding-box; + border-bottom: 1px solid rgba(0, 0, 0, 0.05); +} + +.toast-body { + padding: 0.75rem; +} + +.modal-open { + overflow: hidden; +} + +.modal-open .modal { + overflow-x: hidden; + overflow-y: auto; +} + +.modal { + position: fixed; + top: 0; + left: 0; + z-index: 1050; + display: none; + width: 100%; + height: 100%; + overflow: hidden; + outline: 0; +} + +.modal-dialog { + position: relative; + width: auto; + margin: 0.5rem; + pointer-events: none; +} + +.modal.fade .modal-dialog { + transition: -webkit-transform 0.3s ease-out; + transition: transform 0.3s ease-out; + transition: transform 0.3s ease-out, -webkit-transform 0.3s ease-out; + -webkit-transform: translate(0, -50px); + transform: translate(0, -50px); +} + +@media (prefers-reduced-motion: reduce) { + .modal.fade .modal-dialog { + transition: none; + } +} + +.modal.show .modal-dialog { + -webkit-transform: none; + transform: none; +} + +.modal.modal-static .modal-dialog { + -webkit-transform: scale(1.02); + transform: scale(1.02); +} + +.modal-dialog-scrollable { + display: -ms-flexbox; + display: flex; + max-height: calc(100% - 1rem); +} + +.modal-dialog-scrollable .modal-content { + max-height: calc(100vh - 1rem); + overflow: hidden; +} + +.modal-dialog-scrollable .modal-header, +.modal-dialog-scrollable .modal-footer { + -ms-flex-negative: 0; + flex-shrink: 0; +} + +.modal-dialog-scrollable .modal-body { + overflow-y: auto; +} + +.modal-dialog-centered { + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + min-height: calc(100% - 1rem); +} + +.modal-dialog-centered::before { + display: block; + height: calc(100vh - 1rem); + content: ""; +} + +.modal-dialog-centered.modal-dialog-scrollable { + -ms-flex-direction: column; + flex-direction: column; + -ms-flex-pack: center; + justify-content: center; + height: 100%; +} + +.modal-dialog-centered.modal-dialog-scrollable .modal-content { + max-height: none; +} + +.modal-dialog-centered.modal-dialog-scrollable::before { + content: none; +} + +.modal-content { + position: relative; + display: -ms-flexbox; + display: flex; + -ms-flex-direction: column; + flex-direction: column; + width: 100%; + pointer-events: auto; + background-color: #fff; + background-clip: padding-box; + border: 1px solid rgba(0, 0, 0, 0.2); + border-radius: 0.3rem; + outline: 0; +} + +.modal-backdrop { + position: fixed; + top: 0; + left: 0; + z-index: 1040; + width: 100vw; + height: 100vh; + background-color: #000; +} + +.modal-backdrop.fade { + opacity: 0; +} + +.modal-backdrop.show { + opacity: 0.5; +} + +.modal-header { + display: -ms-flexbox; + display: flex; + -ms-flex-align: start; + align-items: flex-start; + -ms-flex-pack: justify; + justify-content: space-between; + padding: 1rem 1rem; + border-bottom: 1px solid #dee2e6; + border-top-left-radius: calc(0.3rem - 1px); + border-top-right-radius: calc(0.3rem - 1px); +} + +.modal-header .close { + padding: 1rem 1rem; + margin: -1rem -1rem -1rem auto; +} + +.modal-title { + margin-bottom: 0; + line-height: 1.5; +} + +.modal-body { + position: relative; + -ms-flex: 1 1 auto; + flex: 1 1 auto; + padding: 1rem; +} + +.modal-footer { + display: -ms-flexbox; + display: flex; + -ms-flex-wrap: wrap; + flex-wrap: wrap; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: end; + justify-content: flex-end; + padding: 0.75rem; + border-top: 1px solid #dee2e6; + border-bottom-right-radius: calc(0.3rem - 1px); + border-bottom-left-radius: calc(0.3rem - 1px); +} + +.modal-footer > * { + margin: 0.25rem; +} + +.modal-scrollbar-measure { + position: absolute; + top: -9999px; + width: 50px; + height: 50px; + overflow: scroll; +} + +@media (min-width: 576px) { + .modal-dialog { + max-width: 500px; + margin: 1.75rem auto; + } + .modal-dialog-scrollable { + max-height: calc(100% - 3.5rem); + } + .modal-dialog-scrollable .modal-content { + max-height: calc(100vh - 3.5rem); + } + .modal-dialog-centered { + min-height: calc(100% - 3.5rem); + } + .modal-dialog-centered::before { + height: calc(100vh - 3.5rem); + } + .modal-sm { + max-width: 300px; + } +} + +@media (min-width: 992px) { + .modal-lg, + .modal-xl { + max-width: 800px; + } +} + +@media (min-width: 1200px) { + .modal-xl { + max-width: 1140px; + } +} + +.tooltip { + position: absolute; + z-index: 1070; + display: block; + margin: 0; + font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"; + font-style: normal; + font-weight: 400; + line-height: 1.5; + text-align: left; + text-align: start; + text-decoration: none; + text-shadow: none; + text-transform: none; + letter-spacing: normal; + word-break: normal; + word-spacing: normal; + white-space: normal; + line-break: auto; + font-size: 0.875rem; + word-wrap: break-word; + opacity: 0; +} + +.tooltip.show { + opacity: 0.9; +} + +.tooltip .arrow { + position: absolute; + display: block; + width: 0.8rem; + height: 0.4rem; +} + +.tooltip .arrow::before { + position: absolute; + content: ""; + border-color: transparent; + border-style: solid; +} + +.bs-tooltip-top, .bs-tooltip-auto[x-placement^="top"] { + padding: 0.4rem 0; +} + +.bs-tooltip-top .arrow, .bs-tooltip-auto[x-placement^="top"] .arrow { + bottom: 0; +} + +.bs-tooltip-top .arrow::before, .bs-tooltip-auto[x-placement^="top"] .arrow::before { + top: 0; + border-width: 0.4rem 0.4rem 0; + border-top-color: #000; +} + +.bs-tooltip-right, .bs-tooltip-auto[x-placement^="right"] { + padding: 0 0.4rem; +} + +.bs-tooltip-right .arrow, .bs-tooltip-auto[x-placement^="right"] .arrow { + left: 0; + width: 0.4rem; + height: 0.8rem; +} + +.bs-tooltip-right .arrow::before, .bs-tooltip-auto[x-placement^="right"] .arrow::before { + right: 0; + border-width: 0.4rem 0.4rem 0.4rem 0; + border-right-color: #000; +} + +.bs-tooltip-bottom, .bs-tooltip-auto[x-placement^="bottom"] { + padding: 0.4rem 0; +} + +.bs-tooltip-bottom .arrow, .bs-tooltip-auto[x-placement^="bottom"] .arrow { + top: 0; +} + +.bs-tooltip-bottom .arrow::before, .bs-tooltip-auto[x-placement^="bottom"] .arrow::before { + bottom: 0; + border-width: 0 0.4rem 0.4rem; + border-bottom-color: #000; +} + +.bs-tooltip-left, .bs-tooltip-auto[x-placement^="left"] { + padding: 0 0.4rem; +} + +.bs-tooltip-left .arrow, .bs-tooltip-auto[x-placement^="left"] .arrow { + right: 0; + width: 0.4rem; + height: 0.8rem; +} + +.bs-tooltip-left .arrow::before, .bs-tooltip-auto[x-placement^="left"] .arrow::before { + left: 0; + border-width: 0.4rem 0 0.4rem 0.4rem; + border-left-color: #000; +} + +.tooltip-inner { + max-width: 200px; + padding: 0.25rem 0.5rem; + color: #fff; + text-align: center; + background-color: #000; + border-radius: 0.25rem; +} + +.popover { + position: absolute; + top: 0; + left: 0; + z-index: 1060; + display: block; + max-width: 276px; + font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", Roboto, "Helvetica Neue", Arial, "Noto Sans", sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "Noto Color Emoji"; + font-style: normal; + font-weight: 400; + line-height: 1.5; + text-align: left; + text-align: start; + text-decoration: none; + text-shadow: none; + text-transform: none; + letter-spacing: normal; + word-break: normal; + word-spacing: normal; + white-space: normal; + line-break: auto; + font-size: 0.875rem; + word-wrap: break-word; + background-color: #fff; + background-clip: padding-box; + border: 1px solid rgba(0, 0, 0, 0.2); + border-radius: 0.3rem; +} + +.popover .arrow { + position: absolute; + display: block; + width: 1rem; + height: 0.5rem; + margin: 0 0.3rem; +} + +.popover .arrow::before, .popover .arrow::after { + position: absolute; + display: block; + content: ""; + border-color: transparent; + border-style: solid; +} + +.bs-popover-top, .bs-popover-auto[x-placement^="top"] { + margin-bottom: 0.5rem; +} + +.bs-popover-top > .arrow, .bs-popover-auto[x-placement^="top"] > .arrow { + bottom: calc(-0.5rem - 1px); +} + +.bs-popover-top > .arrow::before, .bs-popover-auto[x-placement^="top"] > .arrow::before { + bottom: 0; + border-width: 0.5rem 0.5rem 0; + border-top-color: rgba(0, 0, 0, 0.25); +} + +.bs-popover-top > .arrow::after, .bs-popover-auto[x-placement^="top"] > .arrow::after { + bottom: 1px; + border-width: 0.5rem 0.5rem 0; + border-top-color: #fff; +} + +.bs-popover-right, .bs-popover-auto[x-placement^="right"] { + margin-left: 0.5rem; +} + +.bs-popover-right > .arrow, .bs-popover-auto[x-placement^="right"] > .arrow { + left: calc(-0.5rem - 1px); + width: 0.5rem; + height: 1rem; + margin: 0.3rem 0; +} + +.bs-popover-right > .arrow::before, .bs-popover-auto[x-placement^="right"] > .arrow::before { + left: 0; + border-width: 0.5rem 0.5rem 0.5rem 0; + border-right-color: rgba(0, 0, 0, 0.25); +} + +.bs-popover-right > .arrow::after, .bs-popover-auto[x-placement^="right"] > .arrow::after { + left: 1px; + border-width: 0.5rem 0.5rem 0.5rem 0; + border-right-color: #fff; +} + +.bs-popover-bottom, .bs-popover-auto[x-placement^="bottom"] { + margin-top: 0.5rem; +} + +.bs-popover-bottom > .arrow, .bs-popover-auto[x-placement^="bottom"] > .arrow { + top: calc(-0.5rem - 1px); +} + +.bs-popover-bottom > .arrow::before, .bs-popover-auto[x-placement^="bottom"] > .arrow::before { + top: 0; + border-width: 0 0.5rem 0.5rem 0.5rem; + border-bottom-color: rgba(0, 0, 0, 0.25); +} + +.bs-popover-bottom > .arrow::after, .bs-popover-auto[x-placement^="bottom"] > .arrow::after { + top: 1px; + border-width: 0 0.5rem 0.5rem 0.5rem; + border-bottom-color: #fff; +} + +.bs-popover-bottom .popover-header::before, .bs-popover-auto[x-placement^="bottom"] .popover-header::before { + position: absolute; + top: 0; + left: 50%; + display: block; + width: 1rem; + margin-left: -0.5rem; + content: ""; + border-bottom: 1px solid #f7f7f7; +} + +.bs-popover-left, .bs-popover-auto[x-placement^="left"] { + margin-right: 0.5rem; +} + +.bs-popover-left > .arrow, .bs-popover-auto[x-placement^="left"] > .arrow { + right: calc(-0.5rem - 1px); + width: 0.5rem; + height: 1rem; + margin: 0.3rem 0; +} + +.bs-popover-left > .arrow::before, .bs-popover-auto[x-placement^="left"] > .arrow::before { + right: 0; + border-width: 0.5rem 0 0.5rem 0.5rem; + border-left-color: rgba(0, 0, 0, 0.25); +} + +.bs-popover-left > .arrow::after, .bs-popover-auto[x-placement^="left"] > .arrow::after { + right: 1px; + border-width: 0.5rem 0 0.5rem 0.5rem; + border-left-color: #fff; +} + +.popover-header { + padding: 0.5rem 0.75rem; + margin-bottom: 0; + font-size: 1rem; + background-color: #f7f7f7; + border-bottom: 1px solid #ebebeb; + border-top-left-radius: calc(0.3rem - 1px); + border-top-right-radius: calc(0.3rem - 1px); +} + +.popover-header:empty { + display: none; +} + +.popover-body { + padding: 0.5rem 0.75rem; + color: #212529; +} + +.carousel { + position: relative; +} + +.carousel.pointer-event { + -ms-touch-action: pan-y; + touch-action: pan-y; +} + +.carousel-inner { + position: relative; + width: 100%; + overflow: hidden; +} + +.carousel-inner::after { + display: block; + clear: both; + content: ""; +} + +.carousel-item { + position: relative; + display: none; + float: left; + width: 100%; + margin-right: -100%; + -webkit-backface-visibility: hidden; + backface-visibility: hidden; + transition: -webkit-transform 0.6s ease-in-out; + transition: transform 0.6s ease-in-out; + transition: transform 0.6s ease-in-out, -webkit-transform 0.6s ease-in-out; +} + +@media (prefers-reduced-motion: reduce) { + .carousel-item { + transition: none; + } +} + +.carousel-item.active, +.carousel-item-next, +.carousel-item-prev { + display: block; +} + +.carousel-item-next:not(.carousel-item-left), +.active.carousel-item-right { + -webkit-transform: translateX(100%); + transform: translateX(100%); +} + +.carousel-item-prev:not(.carousel-item-right), +.active.carousel-item-left { + -webkit-transform: translateX(-100%); + transform: translateX(-100%); +} + +.carousel-fade .carousel-item { + opacity: 0; + transition-property: opacity; + -webkit-transform: none; + transform: none; +} + +.carousel-fade .carousel-item.active, +.carousel-fade .carousel-item-next.carousel-item-left, +.carousel-fade .carousel-item-prev.carousel-item-right { + z-index: 1; + opacity: 1; +} + +.carousel-fade .active.carousel-item-left, +.carousel-fade .active.carousel-item-right { + z-index: 0; + opacity: 0; + transition: opacity 0s 0.6s; +} + +@media (prefers-reduced-motion: reduce) { + .carousel-fade .active.carousel-item-left, + .carousel-fade .active.carousel-item-right { + transition: none; + } +} + +.carousel-control-prev, +.carousel-control-next { + position: absolute; + top: 0; + bottom: 0; + z-index: 1; + display: -ms-flexbox; + display: flex; + -ms-flex-align: center; + align-items: center; + -ms-flex-pack: center; + justify-content: center; + width: 15%; + color: #fff; + text-align: center; + opacity: 0.5; + transition: opacity 0.15s ease; +} + +@media (prefers-reduced-motion: reduce) { + .carousel-control-prev, + .carousel-control-next { + transition: none; + } +} + +.carousel-control-prev:hover, .carousel-control-prev:focus, +.carousel-control-next:hover, +.carousel-control-next:focus { + color: #fff; + text-decoration: none; + outline: 0; + opacity: 0.9; +} + +.carousel-control-prev { + left: 0; +} + +.carousel-control-next { + right: 0; +} + +.carousel-control-prev-icon, +.carousel-control-next-icon { + display: inline-block; + width: 20px; + height: 20px; + background: no-repeat 50% / 100% 100%; +} + +.carousel-control-prev-icon { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath d='M5.25 0l-4 4 4 4 1.5-1.5L4.25 4l2.5-2.5L5.25 0z'/%3e%3c/svg%3e"); +} + +.carousel-control-next-icon { + background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath d='M2.75 0l-1.5 1.5L3.75 4l-2.5 2.5L2.75 8l4-4-4-4z'/%3e%3c/svg%3e"); +} + +.carousel-indicators { + position: absolute; + right: 0; + bottom: 0; + left: 0; + z-index: 15; + display: -ms-flexbox; + display: flex; + -ms-flex-pack: center; + justify-content: center; + padding-left: 0; + margin-right: 15%; + margin-left: 15%; + list-style: none; +} + +.carousel-indicators li { + box-sizing: content-box; + -ms-flex: 0 1 auto; + flex: 0 1 auto; + width: 30px; + height: 3px; + margin-right: 3px; + margin-left: 3px; + text-indent: -999px; + cursor: pointer; + background-color: #fff; + background-clip: padding-box; + border-top: 10px solid transparent; + border-bottom: 10px solid transparent; + opacity: .5; + transition: opacity 0.6s ease; +} + +@media (prefers-reduced-motion: reduce) { + .carousel-indicators li { + transition: none; + } +} + +.carousel-indicators .active { + opacity: 1; +} + +.carousel-caption { + position: absolute; + right: 15%; + bottom: 20px; + left: 15%; + z-index: 10; + padding-top: 20px; + padding-bottom: 20px; + color: #fff; + text-align: center; +} + +@-webkit-keyframes spinner-border { + to { + -webkit-transform: rotate(360deg); + transform: rotate(360deg); + } +} + +@keyframes spinner-border { + to { + -webkit-transform: rotate(360deg); + transform: rotate(360deg); + } +} + +.spinner-border { + display: inline-block; + width: 2rem; + height: 2rem; + vertical-align: text-bottom; + border: 0.25em solid currentColor; + border-right-color: transparent; + border-radius: 50%; + -webkit-animation: spinner-border .75s linear infinite; + animation: spinner-border .75s linear infinite; +} + +.spinner-border-sm { + width: 1rem; + height: 1rem; + border-width: 0.2em; +} + +@-webkit-keyframes spinner-grow { + 0% { + -webkit-transform: scale(0); + transform: scale(0); + } + 50% { + opacity: 1; + } +} + +@keyframes spinner-grow { + 0% { + -webkit-transform: scale(0); + transform: scale(0); + } + 50% { + opacity: 1; + } +} + +.spinner-grow { + display: inline-block; + width: 2rem; + height: 2rem; + vertical-align: text-bottom; + background-color: currentColor; + border-radius: 50%; + opacity: 0; + -webkit-animation: spinner-grow .75s linear infinite; + animation: spinner-grow .75s linear infinite; +} + +.spinner-grow-sm { + width: 1rem; + height: 1rem; +} + +.align-baseline { + vertical-align: baseline !important; +} + +.align-top { + vertical-align: top !important; +} + +.align-middle { + vertical-align: middle !important; +} + +.align-bottom { + vertical-align: bottom !important; +} + +.align-text-bottom { + vertical-align: text-bottom !important; +} + +.align-text-top { + vertical-align: text-top !important; +} + +.bg-primary { + background-color: #007bff !important; +} + +a.bg-primary:hover, a.bg-primary:focus, +button.bg-primary:hover, +button.bg-primary:focus { + background-color: #0062cc !important; +} + +.bg-secondary { + background-color: #6c757d !important; +} + +a.bg-secondary:hover, a.bg-secondary:focus, +button.bg-secondary:hover, +button.bg-secondary:focus { + background-color: #545b62 !important; +} + +.bg-success { + background-color: #28a745 !important; +} + +a.bg-success:hover, a.bg-success:focus, +button.bg-success:hover, +button.bg-success:focus { + background-color: #1e7e34 !important; +} + +.bg-info { + background-color: #17a2b8 !important; +} + +a.bg-info:hover, a.bg-info:focus, +button.bg-info:hover, +button.bg-info:focus { + background-color: #117a8b !important; +} + +.bg-warning { + background-color: #ffc107 !important; +} + +a.bg-warning:hover, a.bg-warning:focus, +button.bg-warning:hover, +button.bg-warning:focus { + background-color: #d39e00 !important; +} + +.bg-danger { + background-color: #dc3545 !important; +} + +a.bg-danger:hover, a.bg-danger:focus, +button.bg-danger:hover, +button.bg-danger:focus { + background-color: #bd2130 !important; +} + +.bg-light { + background-color: #f8f9fa !important; +} + +a.bg-light:hover, a.bg-light:focus, +button.bg-light:hover, +button.bg-light:focus { + background-color: #dae0e5 !important; +} + +.bg-dark { + background-color: #343a40 !important; +} + +a.bg-dark:hover, a.bg-dark:focus, +button.bg-dark:hover, +button.bg-dark:focus { + background-color: #1d2124 !important; +} + +.bg-white { + background-color: #fff !important; +} + +.bg-transparent { + background-color: transparent !important; +} + +.border { + border: 1px solid #dee2e6 !important; +} + +.border-top { + border-top: 1px solid #dee2e6 !important; +} + +.border-right { + border-right: 1px solid #dee2e6 !important; +} + +.border-bottom { + border-bottom: 1px solid #dee2e6 !important; +} + +.border-left { + border-left: 1px solid #dee2e6 !important; +} + +.border-0 { + border: 0 !important; +} + +.border-top-0 { + border-top: 0 !important; +} + +.border-right-0 { + border-right: 0 !important; +} + +.border-bottom-0 { + border-bottom: 0 !important; +} + +.border-left-0 { + border-left: 0 !important; +} + +.border-primary { + border-color: #007bff !important; +} + +.border-secondary { + border-color: #6c757d !important; +} + +.border-success { + border-color: #28a745 !important; +} + +.border-info { + border-color: #17a2b8 !important; +} + +.border-warning { + border-color: #ffc107 !important; +} + +.border-danger { + border-color: #dc3545 !important; +} + +.border-light { + border-color: #f8f9fa !important; +} + +.border-dark { + border-color: #343a40 !important; +} + +.border-white { + border-color: #fff !important; +} + +.rounded-sm { + border-radius: 0.2rem !important; +} + +.rounded { + border-radius: 0.25rem !important; +} + +.rounded-top { + border-top-left-radius: 0.25rem !important; + border-top-right-radius: 0.25rem !important; +} + +.rounded-right { + border-top-right-radius: 0.25rem !important; + border-bottom-right-radius: 0.25rem !important; +} + +.rounded-bottom { + border-bottom-right-radius: 0.25rem !important; + border-bottom-left-radius: 0.25rem !important; +} + +.rounded-left { + border-top-left-radius: 0.25rem !important; + border-bottom-left-radius: 0.25rem !important; +} + +.rounded-lg { + border-radius: 0.3rem !important; +} + +.rounded-circle { + border-radius: 50% !important; +} + +.rounded-pill { + border-radius: 50rem !important; +} + +.rounded-0 { + border-radius: 0 !important; +} + +.clearfix::after { + display: block; + clear: both; + content: ""; +} + +.d-none { + display: none !important; +} + +.d-inline { + display: inline !important; +} + +.d-inline-block { + display: inline-block !important; +} + +.d-block { + display: block !important; +} + +.d-table { + display: table !important; +} + +.d-table-row { + display: table-row !important; +} + +.d-table-cell { + display: table-cell !important; +} + +.d-flex { + display: -ms-flexbox !important; + display: flex !important; +} + +.d-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; +} + +@media (min-width: 576px) { + .d-sm-none { + display: none !important; + } + .d-sm-inline { + display: inline !important; + } + .d-sm-inline-block { + display: inline-block !important; + } + .d-sm-block { + display: block !important; + } + .d-sm-table { + display: table !important; + } + .d-sm-table-row { + display: table-row !important; + } + .d-sm-table-cell { + display: table-cell !important; + } + .d-sm-flex { + display: -ms-flexbox !important; + display: flex !important; + } + .d-sm-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; + } +} + +@media (min-width: 768px) { + .d-md-none { + display: none !important; + } + .d-md-inline { + display: inline !important; + } + .d-md-inline-block { + display: inline-block !important; + } + .d-md-block { + display: block !important; + } + .d-md-table { + display: table !important; + } + .d-md-table-row { + display: table-row !important; + } + .d-md-table-cell { + display: table-cell !important; + } + .d-md-flex { + display: -ms-flexbox !important; + display: flex !important; + } + .d-md-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; + } +} + +@media (min-width: 992px) { + .d-lg-none { + display: none !important; + } + .d-lg-inline { + display: inline !important; + } + .d-lg-inline-block { + display: inline-block !important; + } + .d-lg-block { + display: block !important; + } + .d-lg-table { + display: table !important; + } + .d-lg-table-row { + display: table-row !important; + } + .d-lg-table-cell { + display: table-cell !important; + } + .d-lg-flex { + display: -ms-flexbox !important; + display: flex !important; + } + .d-lg-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; + } +} + +@media (min-width: 1200px) { + .d-xl-none { + display: none !important; + } + .d-xl-inline { + display: inline !important; + } + .d-xl-inline-block { + display: inline-block !important; + } + .d-xl-block { + display: block !important; + } + .d-xl-table { + display: table !important; + } + .d-xl-table-row { + display: table-row !important; + } + .d-xl-table-cell { + display: table-cell !important; + } + .d-xl-flex { + display: -ms-flexbox !important; + display: flex !important; + } + .d-xl-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; + } +} + +@media print { + .d-print-none { + display: none !important; + } + .d-print-inline { + display: inline !important; + } + .d-print-inline-block { + display: inline-block !important; + } + .d-print-block { + display: block !important; + } + .d-print-table { + display: table !important; + } + .d-print-table-row { + display: table-row !important; + } + .d-print-table-cell { + display: table-cell !important; + } + .d-print-flex { + display: -ms-flexbox !important; + display: flex !important; + } + .d-print-inline-flex { + display: -ms-inline-flexbox !important; + display: inline-flex !important; + } +} + +.embed-responsive { + position: relative; + display: block; + width: 100%; + padding: 0; + overflow: hidden; +} + +.embed-responsive::before { + display: block; + content: ""; +} + +.embed-responsive .embed-responsive-item, +.embed-responsive iframe, +.embed-responsive embed, +.embed-responsive object, +.embed-responsive video { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 100%; + height: 100%; + border: 0; +} + +.embed-responsive-21by9::before { + padding-top: 42.857143%; +} + +.embed-responsive-16by9::before { + padding-top: 56.25%; +} + +.embed-responsive-4by3::before { + padding-top: 75%; +} + +.embed-responsive-1by1::before { + padding-top: 100%; +} + +.flex-row { + -ms-flex-direction: row !important; + flex-direction: row !important; +} + +.flex-column { + -ms-flex-direction: column !important; + flex-direction: column !important; +} + +.flex-row-reverse { + -ms-flex-direction: row-reverse !important; + flex-direction: row-reverse !important; +} + +.flex-column-reverse { + -ms-flex-direction: column-reverse !important; + flex-direction: column-reverse !important; +} + +.flex-wrap { + -ms-flex-wrap: wrap !important; + flex-wrap: wrap !important; +} + +.flex-nowrap { + -ms-flex-wrap: nowrap !important; + flex-wrap: nowrap !important; +} + +.flex-wrap-reverse { + -ms-flex-wrap: wrap-reverse !important; + flex-wrap: wrap-reverse !important; +} + +.flex-fill { + -ms-flex: 1 1 auto !important; + flex: 1 1 auto !important; +} + +.flex-grow-0 { + -ms-flex-positive: 0 !important; + flex-grow: 0 !important; +} + +.flex-grow-1 { + -ms-flex-positive: 1 !important; + flex-grow: 1 !important; +} + +.flex-shrink-0 { + -ms-flex-negative: 0 !important; + flex-shrink: 0 !important; +} + +.flex-shrink-1 { + -ms-flex-negative: 1 !important; + flex-shrink: 1 !important; +} + +.justify-content-start { + -ms-flex-pack: start !important; + justify-content: flex-start !important; +} + +.justify-content-end { + -ms-flex-pack: end !important; + justify-content: flex-end !important; +} + +.justify-content-center { + -ms-flex-pack: center !important; + justify-content: center !important; +} + +.justify-content-between { + -ms-flex-pack: justify !important; + justify-content: space-between !important; +} + +.justify-content-around { + -ms-flex-pack: distribute !important; + justify-content: space-around !important; +} + +.align-items-start { + -ms-flex-align: start !important; + align-items: flex-start !important; +} + +.align-items-end { + -ms-flex-align: end !important; + align-items: flex-end !important; +} + +.align-items-center { + -ms-flex-align: center !important; + align-items: center !important; +} + +.align-items-baseline { + -ms-flex-align: baseline !important; + align-items: baseline !important; +} + +.align-items-stretch { + -ms-flex-align: stretch !important; + align-items: stretch !important; +} + +.align-content-start { + -ms-flex-line-pack: start !important; + align-content: flex-start !important; +} + +.align-content-end { + -ms-flex-line-pack: end !important; + align-content: flex-end !important; +} + +.align-content-center { + -ms-flex-line-pack: center !important; + align-content: center !important; +} + +.align-content-between { + -ms-flex-line-pack: justify !important; + align-content: space-between !important; +} + +.align-content-around { + -ms-flex-line-pack: distribute !important; + align-content: space-around !important; +} + +.align-content-stretch { + -ms-flex-line-pack: stretch !important; + align-content: stretch !important; +} + +.align-self-auto { + -ms-flex-item-align: auto !important; + align-self: auto !important; +} + +.align-self-start { + -ms-flex-item-align: start !important; + align-self: flex-start !important; +} + +.align-self-end { + -ms-flex-item-align: end !important; + align-self: flex-end !important; +} + +.align-self-center { + -ms-flex-item-align: center !important; + align-self: center !important; +} + +.align-self-baseline { + -ms-flex-item-align: baseline !important; + align-self: baseline !important; +} + +.align-self-stretch { + -ms-flex-item-align: stretch !important; + align-self: stretch !important; +} + +@media (min-width: 576px) { + .flex-sm-row { + -ms-flex-direction: row !important; + flex-direction: row !important; + } + .flex-sm-column { + -ms-flex-direction: column !important; + flex-direction: column !important; + } + .flex-sm-row-reverse { + -ms-flex-direction: row-reverse !important; + flex-direction: row-reverse !important; + } + .flex-sm-column-reverse { + -ms-flex-direction: column-reverse !important; + flex-direction: column-reverse !important; + } + .flex-sm-wrap { + -ms-flex-wrap: wrap !important; + flex-wrap: wrap !important; + } + .flex-sm-nowrap { + -ms-flex-wrap: nowrap !important; + flex-wrap: nowrap !important; + } + .flex-sm-wrap-reverse { + -ms-flex-wrap: wrap-reverse !important; + flex-wrap: wrap-reverse !important; + } + .flex-sm-fill { + -ms-flex: 1 1 auto !important; + flex: 1 1 auto !important; + } + .flex-sm-grow-0 { + -ms-flex-positive: 0 !important; + flex-grow: 0 !important; + } + .flex-sm-grow-1 { + -ms-flex-positive: 1 !important; + flex-grow: 1 !important; + } + .flex-sm-shrink-0 { + -ms-flex-negative: 0 !important; + flex-shrink: 0 !important; + } + .flex-sm-shrink-1 { + -ms-flex-negative: 1 !important; + flex-shrink: 1 !important; + } + .justify-content-sm-start { + -ms-flex-pack: start !important; + justify-content: flex-start !important; + } + .justify-content-sm-end { + -ms-flex-pack: end !important; + justify-content: flex-end !important; + } + .justify-content-sm-center { + -ms-flex-pack: center !important; + justify-content: center !important; + } + .justify-content-sm-between { + -ms-flex-pack: justify !important; + justify-content: space-between !important; + } + .justify-content-sm-around { + -ms-flex-pack: distribute !important; + justify-content: space-around !important; + } + .align-items-sm-start { + -ms-flex-align: start !important; + align-items: flex-start !important; + } + .align-items-sm-end { + -ms-flex-align: end !important; + align-items: flex-end !important; + } + .align-items-sm-center { + -ms-flex-align: center !important; + align-items: center !important; + } + .align-items-sm-baseline { + -ms-flex-align: baseline !important; + align-items: baseline !important; + } + .align-items-sm-stretch { + -ms-flex-align: stretch !important; + align-items: stretch !important; + } + .align-content-sm-start { + -ms-flex-line-pack: start !important; + align-content: flex-start !important; + } + .align-content-sm-end { + -ms-flex-line-pack: end !important; + align-content: flex-end !important; + } + .align-content-sm-center { + -ms-flex-line-pack: center !important; + align-content: center !important; + } + .align-content-sm-between { + -ms-flex-line-pack: justify !important; + align-content: space-between !important; + } + .align-content-sm-around { + -ms-flex-line-pack: distribute !important; + align-content: space-around !important; + } + .align-content-sm-stretch { + -ms-flex-line-pack: stretch !important; + align-content: stretch !important; + } + .align-self-sm-auto { + -ms-flex-item-align: auto !important; + align-self: auto !important; + } + .align-self-sm-start { + -ms-flex-item-align: start !important; + align-self: flex-start !important; + } + .align-self-sm-end { + -ms-flex-item-align: end !important; + align-self: flex-end !important; + } + .align-self-sm-center { + -ms-flex-item-align: center !important; + align-self: center !important; + } + .align-self-sm-baseline { + -ms-flex-item-align: baseline !important; + align-self: baseline !important; + } + .align-self-sm-stretch { + -ms-flex-item-align: stretch !important; + align-self: stretch !important; + } +} + +@media (min-width: 768px) { + .flex-md-row { + -ms-flex-direction: row !important; + flex-direction: row !important; + } + .flex-md-column { + -ms-flex-direction: column !important; + flex-direction: column !important; + } + .flex-md-row-reverse { + -ms-flex-direction: row-reverse !important; + flex-direction: row-reverse !important; + } + .flex-md-column-reverse { + -ms-flex-direction: column-reverse !important; + flex-direction: column-reverse !important; + } + .flex-md-wrap { + -ms-flex-wrap: wrap !important; + flex-wrap: wrap !important; + } + .flex-md-nowrap { + -ms-flex-wrap: nowrap !important; + flex-wrap: nowrap !important; + } + .flex-md-wrap-reverse { + -ms-flex-wrap: wrap-reverse !important; + flex-wrap: wrap-reverse !important; + } + .flex-md-fill { + -ms-flex: 1 1 auto !important; + flex: 1 1 auto !important; + } + .flex-md-grow-0 { + -ms-flex-positive: 0 !important; + flex-grow: 0 !important; + } + .flex-md-grow-1 { + -ms-flex-positive: 1 !important; + flex-grow: 1 !important; + } + .flex-md-shrink-0 { + -ms-flex-negative: 0 !important; + flex-shrink: 0 !important; + } + .flex-md-shrink-1 { + -ms-flex-negative: 1 !important; + flex-shrink: 1 !important; + } + .justify-content-md-start { + -ms-flex-pack: start !important; + justify-content: flex-start !important; + } + .justify-content-md-end { + -ms-flex-pack: end !important; + justify-content: flex-end !important; + } + .justify-content-md-center { + -ms-flex-pack: center !important; + justify-content: center !important; + } + .justify-content-md-between { + -ms-flex-pack: justify !important; + justify-content: space-between !important; + } + .justify-content-md-around { + -ms-flex-pack: distribute !important; + justify-content: space-around !important; + } + .align-items-md-start { + -ms-flex-align: start !important; + align-items: flex-start !important; + } + .align-items-md-end { + -ms-flex-align: end !important; + align-items: flex-end !important; + } + .align-items-md-center { + -ms-flex-align: center !important; + align-items: center !important; + } + .align-items-md-baseline { + -ms-flex-align: baseline !important; + align-items: baseline !important; + } + .align-items-md-stretch { + -ms-flex-align: stretch !important; + align-items: stretch !important; + } + .align-content-md-start { + -ms-flex-line-pack: start !important; + align-content: flex-start !important; + } + .align-content-md-end { + -ms-flex-line-pack: end !important; + align-content: flex-end !important; + } + .align-content-md-center { + -ms-flex-line-pack: center !important; + align-content: center !important; + } + .align-content-md-between { + -ms-flex-line-pack: justify !important; + align-content: space-between !important; + } + .align-content-md-around { + -ms-flex-line-pack: distribute !important; + align-content: space-around !important; + } + .align-content-md-stretch { + -ms-flex-line-pack: stretch !important; + align-content: stretch !important; + } + .align-self-md-auto { + -ms-flex-item-align: auto !important; + align-self: auto !important; + } + .align-self-md-start { + -ms-flex-item-align: start !important; + align-self: flex-start !important; + } + .align-self-md-end { + -ms-flex-item-align: end !important; + align-self: flex-end !important; + } + .align-self-md-center { + -ms-flex-item-align: center !important; + align-self: center !important; + } + .align-self-md-baseline { + -ms-flex-item-align: baseline !important; + align-self: baseline !important; + } + .align-self-md-stretch { + -ms-flex-item-align: stretch !important; + align-self: stretch !important; + } +} + +@media (min-width: 992px) { + .flex-lg-row { + -ms-flex-direction: row !important; + flex-direction: row !important; + } + .flex-lg-column { + -ms-flex-direction: column !important; + flex-direction: column !important; + } + .flex-lg-row-reverse { + -ms-flex-direction: row-reverse !important; + flex-direction: row-reverse !important; + } + .flex-lg-column-reverse { + -ms-flex-direction: column-reverse !important; + flex-direction: column-reverse !important; + } + .flex-lg-wrap { + -ms-flex-wrap: wrap !important; + flex-wrap: wrap !important; + } + .flex-lg-nowrap { + -ms-flex-wrap: nowrap !important; + flex-wrap: nowrap !important; + } + .flex-lg-wrap-reverse { + -ms-flex-wrap: wrap-reverse !important; + flex-wrap: wrap-reverse !important; + } + .flex-lg-fill { + -ms-flex: 1 1 auto !important; + flex: 1 1 auto !important; + } + .flex-lg-grow-0 { + -ms-flex-positive: 0 !important; + flex-grow: 0 !important; + } + .flex-lg-grow-1 { + -ms-flex-positive: 1 !important; + flex-grow: 1 !important; + } + .flex-lg-shrink-0 { + -ms-flex-negative: 0 !important; + flex-shrink: 0 !important; + } + .flex-lg-shrink-1 { + -ms-flex-negative: 1 !important; + flex-shrink: 1 !important; + } + .justify-content-lg-start { + -ms-flex-pack: start !important; + justify-content: flex-start !important; + } + .justify-content-lg-end { + -ms-flex-pack: end !important; + justify-content: flex-end !important; + } + .justify-content-lg-center { + -ms-flex-pack: center !important; + justify-content: center !important; + } + .justify-content-lg-between { + -ms-flex-pack: justify !important; + justify-content: space-between !important; + } + .justify-content-lg-around { + -ms-flex-pack: distribute !important; + justify-content: space-around !important; + } + .align-items-lg-start { + -ms-flex-align: start !important; + align-items: flex-start !important; + } + .align-items-lg-end { + -ms-flex-align: end !important; + align-items: flex-end !important; + } + .align-items-lg-center { + -ms-flex-align: center !important; + align-items: center !important; + } + .align-items-lg-baseline { + -ms-flex-align: baseline !important; + align-items: baseline !important; + } + .align-items-lg-stretch { + -ms-flex-align: stretch !important; + align-items: stretch !important; + } + .align-content-lg-start { + -ms-flex-line-pack: start !important; + align-content: flex-start !important; + } + .align-content-lg-end { + -ms-flex-line-pack: end !important; + align-content: flex-end !important; + } + .align-content-lg-center { + -ms-flex-line-pack: center !important; + align-content: center !important; + } + .align-content-lg-between { + -ms-flex-line-pack: justify !important; + align-content: space-between !important; + } + .align-content-lg-around { + -ms-flex-line-pack: distribute !important; + align-content: space-around !important; + } + .align-content-lg-stretch { + -ms-flex-line-pack: stretch !important; + align-content: stretch !important; + } + .align-self-lg-auto { + -ms-flex-item-align: auto !important; + align-self: auto !important; + } + .align-self-lg-start { + -ms-flex-item-align: start !important; + align-self: flex-start !important; + } + .align-self-lg-end { + -ms-flex-item-align: end !important; + align-self: flex-end !important; + } + .align-self-lg-center { + -ms-flex-item-align: center !important; + align-self: center !important; + } + .align-self-lg-baseline { + -ms-flex-item-align: baseline !important; + align-self: baseline !important; + } + .align-self-lg-stretch { + -ms-flex-item-align: stretch !important; + align-self: stretch !important; + } +} + +@media (min-width: 1200px) { + .flex-xl-row { + -ms-flex-direction: row !important; + flex-direction: row !important; + } + .flex-xl-column { + -ms-flex-direction: column !important; + flex-direction: column !important; + } + .flex-xl-row-reverse { + -ms-flex-direction: row-reverse !important; + flex-direction: row-reverse !important; + } + .flex-xl-column-reverse { + -ms-flex-direction: column-reverse !important; + flex-direction: column-reverse !important; + } + .flex-xl-wrap { + -ms-flex-wrap: wrap !important; + flex-wrap: wrap !important; + } + .flex-xl-nowrap { + -ms-flex-wrap: nowrap !important; + flex-wrap: nowrap !important; + } + .flex-xl-wrap-reverse { + -ms-flex-wrap: wrap-reverse !important; + flex-wrap: wrap-reverse !important; + } + .flex-xl-fill { + -ms-flex: 1 1 auto !important; + flex: 1 1 auto !important; + } + .flex-xl-grow-0 { + -ms-flex-positive: 0 !important; + flex-grow: 0 !important; + } + .flex-xl-grow-1 { + -ms-flex-positive: 1 !important; + flex-grow: 1 !important; + } + .flex-xl-shrink-0 { + -ms-flex-negative: 0 !important; + flex-shrink: 0 !important; + } + .flex-xl-shrink-1 { + -ms-flex-negative: 1 !important; + flex-shrink: 1 !important; + } + .justify-content-xl-start { + -ms-flex-pack: start !important; + justify-content: flex-start !important; + } + .justify-content-xl-end { + -ms-flex-pack: end !important; + justify-content: flex-end !important; + } + .justify-content-xl-center { + -ms-flex-pack: center !important; + justify-content: center !important; + } + .justify-content-xl-between { + -ms-flex-pack: justify !important; + justify-content: space-between !important; + } + .justify-content-xl-around { + -ms-flex-pack: distribute !important; + justify-content: space-around !important; + } + .align-items-xl-start { + -ms-flex-align: start !important; + align-items: flex-start !important; + } + .align-items-xl-end { + -ms-flex-align: end !important; + align-items: flex-end !important; + } + .align-items-xl-center { + -ms-flex-align: center !important; + align-items: center !important; + } + .align-items-xl-baseline { + -ms-flex-align: baseline !important; + align-items: baseline !important; + } + .align-items-xl-stretch { + -ms-flex-align: stretch !important; + align-items: stretch !important; + } + .align-content-xl-start { + -ms-flex-line-pack: start !important; + align-content: flex-start !important; + } + .align-content-xl-end { + -ms-flex-line-pack: end !important; + align-content: flex-end !important; + } + .align-content-xl-center { + -ms-flex-line-pack: center !important; + align-content: center !important; + } + .align-content-xl-between { + -ms-flex-line-pack: justify !important; + align-content: space-between !important; + } + .align-content-xl-around { + -ms-flex-line-pack: distribute !important; + align-content: space-around !important; + } + .align-content-xl-stretch { + -ms-flex-line-pack: stretch !important; + align-content: stretch !important; + } + .align-self-xl-auto { + -ms-flex-item-align: auto !important; + align-self: auto !important; + } + .align-self-xl-start { + -ms-flex-item-align: start !important; + align-self: flex-start !important; + } + .align-self-xl-end { + -ms-flex-item-align: end !important; + align-self: flex-end !important; + } + .align-self-xl-center { + -ms-flex-item-align: center !important; + align-self: center !important; + } + .align-self-xl-baseline { + -ms-flex-item-align: baseline !important; + align-self: baseline !important; + } + .align-self-xl-stretch { + -ms-flex-item-align: stretch !important; + align-self: stretch !important; + } +} + +.float-left { + float: left !important; +} + +.float-right { + float: right !important; +} + +.float-none { + float: none !important; +} + +@media (min-width: 576px) { + .float-sm-left { + float: left !important; + } + .float-sm-right { + float: right !important; + } + .float-sm-none { + float: none !important; + } +} + +@media (min-width: 768px) { + .float-md-left { + float: left !important; + } + .float-md-right { + float: right !important; + } + .float-md-none { + float: none !important; + } +} + +@media (min-width: 992px) { + .float-lg-left { + float: left !important; + } + .float-lg-right { + float: right !important; + } + .float-lg-none { + float: none !important; + } +} + +@media (min-width: 1200px) { + .float-xl-left { + float: left !important; + } + .float-xl-right { + float: right !important; + } + .float-xl-none { + float: none !important; + } +} + +.overflow-auto { + overflow: auto !important; +} + +.overflow-hidden { + overflow: hidden !important; +} + +.position-static { + position: static !important; +} + +.position-relative { + position: relative !important; +} + +.position-absolute { + position: absolute !important; +} + +.position-fixed { + position: fixed !important; +} + +.position-sticky { + position: -webkit-sticky !important; + position: sticky !important; +} + +.fixed-top { + position: fixed; + top: 0; + right: 0; + left: 0; + z-index: 1030; +} + +.fixed-bottom { + position: fixed; + right: 0; + bottom: 0; + left: 0; + z-index: 1030; +} + +@supports ((position: -webkit-sticky) or (position: sticky)) { + .sticky-top { + position: -webkit-sticky; + position: sticky; + top: 0; + z-index: 1020; + } +} + +.sr-only { + position: absolute; + width: 1px; + height: 1px; + padding: 0; + margin: -1px; + overflow: hidden; + clip: rect(0, 0, 0, 0); + white-space: nowrap; + border: 0; +} + +.sr-only-focusable:active, .sr-only-focusable:focus { + position: static; + width: auto; + height: auto; + overflow: visible; + clip: auto; + white-space: normal; +} + +.shadow-sm { + box-shadow: 0 0.125rem 0.25rem rgba(0, 0, 0, 0.075) !important; +} + +.shadow { + box-shadow: 0 0.5rem 1rem rgba(0, 0, 0, 0.15) !important; +} + +.shadow-lg { + box-shadow: 0 1rem 3rem rgba(0, 0, 0, 0.175) !important; +} + +.shadow-none { + box-shadow: none !important; +} + +.w-25 { + width: 25% !important; +} + +.w-50 { + width: 50% !important; +} + +.w-75 { + width: 75% !important; +} + +.w-100 { + width: 100% !important; +} + +.w-auto { + width: auto !important; +} + +.h-25 { + height: 25% !important; +} + +.h-50 { + height: 50% !important; +} + +.h-75 { + height: 75% !important; +} + +.h-100 { + height: 100% !important; +} + +.h-auto { + height: auto !important; +} + +.mw-100 { + max-width: 100% !important; +} + +.mh-100 { + max-height: 100% !important; +} + +.min-vw-100 { + min-width: 100vw !important; +} + +.min-vh-100 { + min-height: 100vh !important; +} + +.vw-100 { + width: 100vw !important; +} + +.vh-100 { + height: 100vh !important; +} + +.stretched-link::after { + position: absolute; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1; + pointer-events: auto; + content: ""; + background-color: rgba(0, 0, 0, 0); +} + +.m-0 { + margin: 0 !important; +} + +.mt-0, +.my-0 { + margin-top: 0 !important; +} + +.mr-0, +.mx-0 { + margin-right: 0 !important; +} + +.mb-0, +.my-0 { + margin-bottom: 0 !important; +} + +.ml-0, +.mx-0 { + margin-left: 0 !important; +} + +.m-1 { + margin: 0.25rem !important; +} + +.mt-1, +.my-1 { + margin-top: 0.25rem !important; +} + +.mr-1, +.mx-1 { + margin-right: 0.25rem !important; +} + +.mb-1, +.my-1 { + margin-bottom: 0.25rem !important; +} + +.ml-1, +.mx-1 { + margin-left: 0.25rem !important; +} + +.m-2 { + margin: 0.5rem !important; +} + +.mt-2, +.my-2 { + margin-top: 0.5rem !important; +} + +.mr-2, +.mx-2 { + margin-right: 0.5rem !important; +} + +.mb-2, +.my-2 { + margin-bottom: 0.5rem !important; +} + +.ml-2, +.mx-2 { + margin-left: 0.5rem !important; +} + +.m-3 { + margin: 1rem !important; +} + +.mt-3, +.my-3 { + margin-top: 1rem !important; +} + +.mr-3, +.mx-3 { + margin-right: 1rem !important; +} + +.mb-3, +.my-3 { + margin-bottom: 1rem !important; +} + +.ml-3, +.mx-3 { + margin-left: 1rem !important; +} + +.m-4 { + margin: 1.5rem !important; +} + +.mt-4, +.my-4 { + margin-top: 1.5rem !important; +} + +.mr-4, +.mx-4 { + margin-right: 1.5rem !important; +} + +.mb-4, +.my-4 { + margin-bottom: 1.5rem !important; +} + +.ml-4, +.mx-4 { + margin-left: 1.5rem !important; +} + +.m-5 { + margin: 3rem !important; +} + +.mt-5, +.my-5 { + margin-top: 3rem !important; +} + +.mr-5, +.mx-5 { + margin-right: 3rem !important; +} + +.mb-5, +.my-5 { + margin-bottom: 3rem !important; +} + +.ml-5, +.mx-5 { + margin-left: 3rem !important; +} + +.p-0 { + padding: 0 !important; +} + +.pt-0, +.py-0 { + padding-top: 0 !important; +} + +.pr-0, +.px-0 { + padding-right: 0 !important; +} + +.pb-0, +.py-0 { + padding-bottom: 0 !important; +} + +.pl-0, +.px-0 { + padding-left: 0 !important; +} + +.p-1 { + padding: 0.25rem !important; +} + +.pt-1, +.py-1 { + padding-top: 0.25rem !important; +} + +.pr-1, +.px-1 { + padding-right: 0.25rem !important; +} + +.pb-1, +.py-1 { + padding-bottom: 0.25rem !important; +} + +.pl-1, +.px-1 { + padding-left: 0.25rem !important; +} + +.p-2 { + padding: 0.5rem !important; +} + +.pt-2, +.py-2 { + padding-top: 0.5rem !important; +} + +.pr-2, +.px-2 { + padding-right: 0.5rem !important; +} + +.pb-2, +.py-2 { + padding-bottom: 0.5rem !important; +} + +.pl-2, +.px-2 { + padding-left: 0.5rem !important; +} + +.p-3 { + padding: 1rem !important; +} + +.pt-3, +.py-3 { + padding-top: 1rem !important; +} + +.pr-3, +.px-3 { + padding-right: 1rem !important; +} + +.pb-3, +.py-3 { + padding-bottom: 1rem !important; +} + +.pl-3, +.px-3 { + padding-left: 1rem !important; +} + +.p-4 { + padding: 1.5rem !important; +} + +.pt-4, +.py-4 { + padding-top: 1.5rem !important; +} + +.pr-4, +.px-4 { + padding-right: 1.5rem !important; +} + +.pb-4, +.py-4 { + padding-bottom: 1.5rem !important; +} + +.pl-4, +.px-4 { + padding-left: 1.5rem !important; +} + +.p-5 { + padding: 3rem !important; +} + +.pt-5, +.py-5 { + padding-top: 3rem !important; +} + +.pr-5, +.px-5 { + padding-right: 3rem !important; +} + +.pb-5, +.py-5 { + padding-bottom: 3rem !important; +} + +.pl-5, +.px-5 { + padding-left: 3rem !important; +} + +.m-n1 { + margin: -0.25rem !important; +} + +.mt-n1, +.my-n1 { + margin-top: -0.25rem !important; +} + +.mr-n1, +.mx-n1 { + margin-right: -0.25rem !important; +} + +.mb-n1, +.my-n1 { + margin-bottom: -0.25rem !important; +} + +.ml-n1, +.mx-n1 { + margin-left: -0.25rem !important; +} + +.m-n2 { + margin: -0.5rem !important; +} + +.mt-n2, +.my-n2 { + margin-top: -0.5rem !important; +} + +.mr-n2, +.mx-n2 { + margin-right: -0.5rem !important; +} + +.mb-n2, +.my-n2 { + margin-bottom: -0.5rem !important; +} + +.ml-n2, +.mx-n2 { + margin-left: -0.5rem !important; +} + +.m-n3 { + margin: -1rem !important; +} + +.mt-n3, +.my-n3 { + margin-top: -1rem !important; +} + +.mr-n3, +.mx-n3 { + margin-right: -1rem !important; +} + +.mb-n3, +.my-n3 { + margin-bottom: -1rem !important; +} + +.ml-n3, +.mx-n3 { + margin-left: -1rem !important; +} + +.m-n4 { + margin: -1.5rem !important; +} + +.mt-n4, +.my-n4 { + margin-top: -1.5rem !important; +} + +.mr-n4, +.mx-n4 { + margin-right: -1.5rem !important; +} + +.mb-n4, +.my-n4 { + margin-bottom: -1.5rem !important; +} + +.ml-n4, +.mx-n4 { + margin-left: -1.5rem !important; +} + +.m-n5 { + margin: -3rem !important; +} + +.mt-n5, +.my-n5 { + margin-top: -3rem !important; +} + +.mr-n5, +.mx-n5 { + margin-right: -3rem !important; +} + +.mb-n5, +.my-n5 { + margin-bottom: -3rem !important; +} + +.ml-n5, +.mx-n5 { + margin-left: -3rem !important; +} + +.m-auto { + margin: auto !important; +} + +.mt-auto, +.my-auto { + margin-top: auto !important; +} + +.mr-auto, +.mx-auto { + margin-right: auto !important; +} + +.mb-auto, +.my-auto { + margin-bottom: auto !important; +} + +.ml-auto, +.mx-auto { + margin-left: auto !important; +} + +@media (min-width: 576px) { + .m-sm-0 { + margin: 0 !important; + } + .mt-sm-0, + .my-sm-0 { + margin-top: 0 !important; + } + .mr-sm-0, + .mx-sm-0 { + margin-right: 0 !important; + } + .mb-sm-0, + .my-sm-0 { + margin-bottom: 0 !important; + } + .ml-sm-0, + .mx-sm-0 { + margin-left: 0 !important; + } + .m-sm-1 { + margin: 0.25rem !important; + } + .mt-sm-1, + .my-sm-1 { + margin-top: 0.25rem !important; + } + .mr-sm-1, + .mx-sm-1 { + margin-right: 0.25rem !important; + } + .mb-sm-1, + .my-sm-1 { + margin-bottom: 0.25rem !important; + } + .ml-sm-1, + .mx-sm-1 { + margin-left: 0.25rem !important; + } + .m-sm-2 { + margin: 0.5rem !important; + } + .mt-sm-2, + .my-sm-2 { + margin-top: 0.5rem !important; + } + .mr-sm-2, + .mx-sm-2 { + margin-right: 0.5rem !important; + } + .mb-sm-2, + .my-sm-2 { + margin-bottom: 0.5rem !important; + } + .ml-sm-2, + .mx-sm-2 { + margin-left: 0.5rem !important; + } + .m-sm-3 { + margin: 1rem !important; + } + .mt-sm-3, + .my-sm-3 { + margin-top: 1rem !important; + } + .mr-sm-3, + .mx-sm-3 { + margin-right: 1rem !important; + } + .mb-sm-3, + .my-sm-3 { + margin-bottom: 1rem !important; + } + .ml-sm-3, + .mx-sm-3 { + margin-left: 1rem !important; + } + .m-sm-4 { + margin: 1.5rem !important; + } + .mt-sm-4, + .my-sm-4 { + margin-top: 1.5rem !important; + } + .mr-sm-4, + .mx-sm-4 { + margin-right: 1.5rem !important; + } + .mb-sm-4, + .my-sm-4 { + margin-bottom: 1.5rem !important; + } + .ml-sm-4, + .mx-sm-4 { + margin-left: 1.5rem !important; + } + .m-sm-5 { + margin: 3rem !important; + } + .mt-sm-5, + .my-sm-5 { + margin-top: 3rem !important; + } + .mr-sm-5, + .mx-sm-5 { + margin-right: 3rem !important; + } + .mb-sm-5, + .my-sm-5 { + margin-bottom: 3rem !important; + } + .ml-sm-5, + .mx-sm-5 { + margin-left: 3rem !important; + } + .p-sm-0 { + padding: 0 !important; + } + .pt-sm-0, + .py-sm-0 { + padding-top: 0 !important; + } + .pr-sm-0, + .px-sm-0 { + padding-right: 0 !important; + } + .pb-sm-0, + .py-sm-0 { + padding-bottom: 0 !important; + } + .pl-sm-0, + .px-sm-0 { + padding-left: 0 !important; + } + .p-sm-1 { + padding: 0.25rem !important; + } + .pt-sm-1, + .py-sm-1 { + padding-top: 0.25rem !important; + } + .pr-sm-1, + .px-sm-1 { + padding-right: 0.25rem !important; + } + .pb-sm-1, + .py-sm-1 { + padding-bottom: 0.25rem !important; + } + .pl-sm-1, + .px-sm-1 { + padding-left: 0.25rem !important; + } + .p-sm-2 { + padding: 0.5rem !important; + } + .pt-sm-2, + .py-sm-2 { + padding-top: 0.5rem !important; + } + .pr-sm-2, + .px-sm-2 { + padding-right: 0.5rem !important; + } + .pb-sm-2, + .py-sm-2 { + padding-bottom: 0.5rem !important; + } + .pl-sm-2, + .px-sm-2 { + padding-left: 0.5rem !important; + } + .p-sm-3 { + padding: 1rem !important; + } + .pt-sm-3, + .py-sm-3 { + padding-top: 1rem !important; + } + .pr-sm-3, + .px-sm-3 { + padding-right: 1rem !important; + } + .pb-sm-3, + .py-sm-3 { + padding-bottom: 1rem !important; + } + .pl-sm-3, + .px-sm-3 { + padding-left: 1rem !important; + } + .p-sm-4 { + padding: 1.5rem !important; + } + .pt-sm-4, + .py-sm-4 { + padding-top: 1.5rem !important; + } + .pr-sm-4, + .px-sm-4 { + padding-right: 1.5rem !important; + } + .pb-sm-4, + .py-sm-4 { + padding-bottom: 1.5rem !important; + } + .pl-sm-4, + .px-sm-4 { + padding-left: 1.5rem !important; + } + .p-sm-5 { + padding: 3rem !important; + } + .pt-sm-5, + .py-sm-5 { + padding-top: 3rem !important; + } + .pr-sm-5, + .px-sm-5 { + padding-right: 3rem !important; + } + .pb-sm-5, + .py-sm-5 { + padding-bottom: 3rem !important; + } + .pl-sm-5, + .px-sm-5 { + padding-left: 3rem !important; + } + .m-sm-n1 { + margin: -0.25rem !important; + } + .mt-sm-n1, + .my-sm-n1 { + margin-top: -0.25rem !important; + } + .mr-sm-n1, + .mx-sm-n1 { + margin-right: -0.25rem !important; + } + .mb-sm-n1, + .my-sm-n1 { + margin-bottom: -0.25rem !important; + } + .ml-sm-n1, + .mx-sm-n1 { + margin-left: -0.25rem !important; + } + .m-sm-n2 { + margin: -0.5rem !important; + } + .mt-sm-n2, + .my-sm-n2 { + margin-top: -0.5rem !important; + } + .mr-sm-n2, + .mx-sm-n2 { + margin-right: -0.5rem !important; + } + .mb-sm-n2, + .my-sm-n2 { + margin-bottom: -0.5rem !important; + } + .ml-sm-n2, + .mx-sm-n2 { + margin-left: -0.5rem !important; + } + .m-sm-n3 { + margin: -1rem !important; + } + .mt-sm-n3, + .my-sm-n3 { + margin-top: -1rem !important; + } + .mr-sm-n3, + .mx-sm-n3 { + margin-right: -1rem !important; + } + .mb-sm-n3, + .my-sm-n3 { + margin-bottom: -1rem !important; + } + .ml-sm-n3, + .mx-sm-n3 { + margin-left: -1rem !important; + } + .m-sm-n4 { + margin: -1.5rem !important; + } + .mt-sm-n4, + .my-sm-n4 { + margin-top: -1.5rem !important; + } + .mr-sm-n4, + .mx-sm-n4 { + margin-right: -1.5rem !important; + } + .mb-sm-n4, + .my-sm-n4 { + margin-bottom: -1.5rem !important; + } + .ml-sm-n4, + .mx-sm-n4 { + margin-left: -1.5rem !important; + } + .m-sm-n5 { + margin: -3rem !important; + } + .mt-sm-n5, + .my-sm-n5 { + margin-top: -3rem !important; + } + .mr-sm-n5, + .mx-sm-n5 { + margin-right: -3rem !important; + } + .mb-sm-n5, + .my-sm-n5 { + margin-bottom: -3rem !important; + } + .ml-sm-n5, + .mx-sm-n5 { + margin-left: -3rem !important; + } + .m-sm-auto { + margin: auto !important; + } + .mt-sm-auto, + .my-sm-auto { + margin-top: auto !important; + } + .mr-sm-auto, + .mx-sm-auto { + margin-right: auto !important; + } + .mb-sm-auto, + .my-sm-auto { + margin-bottom: auto !important; + } + .ml-sm-auto, + .mx-sm-auto { + margin-left: auto !important; + } +} + +@media (min-width: 768px) { + .m-md-0 { + margin: 0 !important; + } + .mt-md-0, + .my-md-0 { + margin-top: 0 !important; + } + .mr-md-0, + .mx-md-0 { + margin-right: 0 !important; + } + .mb-md-0, + .my-md-0 { + margin-bottom: 0 !important; + } + .ml-md-0, + .mx-md-0 { + margin-left: 0 !important; + } + .m-md-1 { + margin: 0.25rem !important; + } + .mt-md-1, + .my-md-1 { + margin-top: 0.25rem !important; + } + .mr-md-1, + .mx-md-1 { + margin-right: 0.25rem !important; + } + .mb-md-1, + .my-md-1 { + margin-bottom: 0.25rem !important; + } + .ml-md-1, + .mx-md-1 { + margin-left: 0.25rem !important; + } + .m-md-2 { + margin: 0.5rem !important; + } + .mt-md-2, + .my-md-2 { + margin-top: 0.5rem !important; + } + .mr-md-2, + .mx-md-2 { + margin-right: 0.5rem !important; + } + .mb-md-2, + .my-md-2 { + margin-bottom: 0.5rem !important; + } + .ml-md-2, + .mx-md-2 { + margin-left: 0.5rem !important; + } + .m-md-3 { + margin: 1rem !important; + } + .mt-md-3, + .my-md-3 { + margin-top: 1rem !important; + } + .mr-md-3, + .mx-md-3 { + margin-right: 1rem !important; + } + .mb-md-3, + .my-md-3 { + margin-bottom: 1rem !important; + } + .ml-md-3, + .mx-md-3 { + margin-left: 1rem !important; + } + .m-md-4 { + margin: 1.5rem !important; + } + .mt-md-4, + .my-md-4 { + margin-top: 1.5rem !important; + } + .mr-md-4, + .mx-md-4 { + margin-right: 1.5rem !important; + } + .mb-md-4, + .my-md-4 { + margin-bottom: 1.5rem !important; + } + .ml-md-4, + .mx-md-4 { + margin-left: 1.5rem !important; + } + .m-md-5 { + margin: 3rem !important; + } + .mt-md-5, + .my-md-5 { + margin-top: 3rem !important; + } + .mr-md-5, + .mx-md-5 { + margin-right: 3rem !important; + } + .mb-md-5, + .my-md-5 { + margin-bottom: 3rem !important; + } + .ml-md-5, + .mx-md-5 { + margin-left: 3rem !important; + } + .p-md-0 { + padding: 0 !important; + } + .pt-md-0, + .py-md-0 { + padding-top: 0 !important; + } + .pr-md-0, + .px-md-0 { + padding-right: 0 !important; + } + .pb-md-0, + .py-md-0 { + padding-bottom: 0 !important; + } + .pl-md-0, + .px-md-0 { + padding-left: 0 !important; + } + .p-md-1 { + padding: 0.25rem !important; + } + .pt-md-1, + .py-md-1 { + padding-top: 0.25rem !important; + } + .pr-md-1, + .px-md-1 { + padding-right: 0.25rem !important; + } + .pb-md-1, + .py-md-1 { + padding-bottom: 0.25rem !important; + } + .pl-md-1, + .px-md-1 { + padding-left: 0.25rem !important; + } + .p-md-2 { + padding: 0.5rem !important; + } + .pt-md-2, + .py-md-2 { + padding-top: 0.5rem !important; + } + .pr-md-2, + .px-md-2 { + padding-right: 0.5rem !important; + } + .pb-md-2, + .py-md-2 { + padding-bottom: 0.5rem !important; + } + .pl-md-2, + .px-md-2 { + padding-left: 0.5rem !important; + } + .p-md-3 { + padding: 1rem !important; + } + .pt-md-3, + .py-md-3 { + padding-top: 1rem !important; + } + .pr-md-3, + .px-md-3 { + padding-right: 1rem !important; + } + .pb-md-3, + .py-md-3 { + padding-bottom: 1rem !important; + } + .pl-md-3, + .px-md-3 { + padding-left: 1rem !important; + } + .p-md-4 { + padding: 1.5rem !important; + } + .pt-md-4, + .py-md-4 { + padding-top: 1.5rem !important; + } + .pr-md-4, + .px-md-4 { + padding-right: 1.5rem !important; + } + .pb-md-4, + .py-md-4 { + padding-bottom: 1.5rem !important; + } + .pl-md-4, + .px-md-4 { + padding-left: 1.5rem !important; + } + .p-md-5 { + padding: 3rem !important; + } + .pt-md-5, + .py-md-5 { + padding-top: 3rem !important; + } + .pr-md-5, + .px-md-5 { + padding-right: 3rem !important; + } + .pb-md-5, + .py-md-5 { + padding-bottom: 3rem !important; + } + .pl-md-5, + .px-md-5 { + padding-left: 3rem !important; + } + .m-md-n1 { + margin: -0.25rem !important; + } + .mt-md-n1, + .my-md-n1 { + margin-top: -0.25rem !important; + } + .mr-md-n1, + .mx-md-n1 { + margin-right: -0.25rem !important; + } + .mb-md-n1, + .my-md-n1 { + margin-bottom: -0.25rem !important; + } + .ml-md-n1, + .mx-md-n1 { + margin-left: -0.25rem !important; + } + .m-md-n2 { + margin: -0.5rem !important; + } + .mt-md-n2, + .my-md-n2 { + margin-top: -0.5rem !important; + } + .mr-md-n2, + .mx-md-n2 { + margin-right: -0.5rem !important; + } + .mb-md-n2, + .my-md-n2 { + margin-bottom: -0.5rem !important; + } + .ml-md-n2, + .mx-md-n2 { + margin-left: -0.5rem !important; + } + .m-md-n3 { + margin: -1rem !important; + } + .mt-md-n3, + .my-md-n3 { + margin-top: -1rem !important; + } + .mr-md-n3, + .mx-md-n3 { + margin-right: -1rem !important; + } + .mb-md-n3, + .my-md-n3 { + margin-bottom: -1rem !important; + } + .ml-md-n3, + .mx-md-n3 { + margin-left: -1rem !important; + } + .m-md-n4 { + margin: -1.5rem !important; + } + .mt-md-n4, + .my-md-n4 { + margin-top: -1.5rem !important; + } + .mr-md-n4, + .mx-md-n4 { + margin-right: -1.5rem !important; + } + .mb-md-n4, + .my-md-n4 { + margin-bottom: -1.5rem !important; + } + .ml-md-n4, + .mx-md-n4 { + margin-left: -1.5rem !important; + } + .m-md-n5 { + margin: -3rem !important; + } + .mt-md-n5, + .my-md-n5 { + margin-top: -3rem !important; + } + .mr-md-n5, + .mx-md-n5 { + margin-right: -3rem !important; + } + .mb-md-n5, + .my-md-n5 { + margin-bottom: -3rem !important; + } + .ml-md-n5, + .mx-md-n5 { + margin-left: -3rem !important; + } + .m-md-auto { + margin: auto !important; + } + .mt-md-auto, + .my-md-auto { + margin-top: auto !important; + } + .mr-md-auto, + .mx-md-auto { + margin-right: auto !important; + } + .mb-md-auto, + .my-md-auto { + margin-bottom: auto !important; + } + .ml-md-auto, + .mx-md-auto { + margin-left: auto !important; + } +} + +@media (min-width: 992px) { + .m-lg-0 { + margin: 0 !important; + } + .mt-lg-0, + .my-lg-0 { + margin-top: 0 !important; + } + .mr-lg-0, + .mx-lg-0 { + margin-right: 0 !important; + } + .mb-lg-0, + .my-lg-0 { + margin-bottom: 0 !important; + } + .ml-lg-0, + .mx-lg-0 { + margin-left: 0 !important; + } + .m-lg-1 { + margin: 0.25rem !important; + } + .mt-lg-1, + .my-lg-1 { + margin-top: 0.25rem !important; + } + .mr-lg-1, + .mx-lg-1 { + margin-right: 0.25rem !important; + } + .mb-lg-1, + .my-lg-1 { + margin-bottom: 0.25rem !important; + } + .ml-lg-1, + .mx-lg-1 { + margin-left: 0.25rem !important; + } + .m-lg-2 { + margin: 0.5rem !important; + } + .mt-lg-2, + .my-lg-2 { + margin-top: 0.5rem !important; + } + .mr-lg-2, + .mx-lg-2 { + margin-right: 0.5rem !important; + } + .mb-lg-2, + .my-lg-2 { + margin-bottom: 0.5rem !important; + } + .ml-lg-2, + .mx-lg-2 { + margin-left: 0.5rem !important; + } + .m-lg-3 { + margin: 1rem !important; + } + .mt-lg-3, + .my-lg-3 { + margin-top: 1rem !important; + } + .mr-lg-3, + .mx-lg-3 { + margin-right: 1rem !important; + } + .mb-lg-3, + .my-lg-3 { + margin-bottom: 1rem !important; + } + .ml-lg-3, + .mx-lg-3 { + margin-left: 1rem !important; + } + .m-lg-4 { + margin: 1.5rem !important; + } + .mt-lg-4, + .my-lg-4 { + margin-top: 1.5rem !important; + } + .mr-lg-4, + .mx-lg-4 { + margin-right: 1.5rem !important; + } + .mb-lg-4, + .my-lg-4 { + margin-bottom: 1.5rem !important; + } + .ml-lg-4, + .mx-lg-4 { + margin-left: 1.5rem !important; + } + .m-lg-5 { + margin: 3rem !important; + } + .mt-lg-5, + .my-lg-5 { + margin-top: 3rem !important; + } + .mr-lg-5, + .mx-lg-5 { + margin-right: 3rem !important; + } + .mb-lg-5, + .my-lg-5 { + margin-bottom: 3rem !important; + } + .ml-lg-5, + .mx-lg-5 { + margin-left: 3rem !important; + } + .p-lg-0 { + padding: 0 !important; + } + .pt-lg-0, + .py-lg-0 { + padding-top: 0 !important; + } + .pr-lg-0, + .px-lg-0 { + padding-right: 0 !important; + } + .pb-lg-0, + .py-lg-0 { + padding-bottom: 0 !important; + } + .pl-lg-0, + .px-lg-0 { + padding-left: 0 !important; + } + .p-lg-1 { + padding: 0.25rem !important; + } + .pt-lg-1, + .py-lg-1 { + padding-top: 0.25rem !important; + } + .pr-lg-1, + .px-lg-1 { + padding-right: 0.25rem !important; + } + .pb-lg-1, + .py-lg-1 { + padding-bottom: 0.25rem !important; + } + .pl-lg-1, + .px-lg-1 { + padding-left: 0.25rem !important; + } + .p-lg-2 { + padding: 0.5rem !important; + } + .pt-lg-2, + .py-lg-2 { + padding-top: 0.5rem !important; + } + .pr-lg-2, + .px-lg-2 { + padding-right: 0.5rem !important; + } + .pb-lg-2, + .py-lg-2 { + padding-bottom: 0.5rem !important; + } + .pl-lg-2, + .px-lg-2 { + padding-left: 0.5rem !important; + } + .p-lg-3 { + padding: 1rem !important; + } + .pt-lg-3, + .py-lg-3 { + padding-top: 1rem !important; + } + .pr-lg-3, + .px-lg-3 { + padding-right: 1rem !important; + } + .pb-lg-3, + .py-lg-3 { + padding-bottom: 1rem !important; + } + .pl-lg-3, + .px-lg-3 { + padding-left: 1rem !important; + } + .p-lg-4 { + padding: 1.5rem !important; + } + .pt-lg-4, + .py-lg-4 { + padding-top: 1.5rem !important; + } + .pr-lg-4, + .px-lg-4 { + padding-right: 1.5rem !important; + } + .pb-lg-4, + .py-lg-4 { + padding-bottom: 1.5rem !important; + } + .pl-lg-4, + .px-lg-4 { + padding-left: 1.5rem !important; + } + .p-lg-5 { + padding: 3rem !important; + } + .pt-lg-5, + .py-lg-5 { + padding-top: 3rem !important; + } + .pr-lg-5, + .px-lg-5 { + padding-right: 3rem !important; + } + .pb-lg-5, + .py-lg-5 { + padding-bottom: 3rem !important; + } + .pl-lg-5, + .px-lg-5 { + padding-left: 3rem !important; + } + .m-lg-n1 { + margin: -0.25rem !important; + } + .mt-lg-n1, + .my-lg-n1 { + margin-top: -0.25rem !important; + } + .mr-lg-n1, + .mx-lg-n1 { + margin-right: -0.25rem !important; + } + .mb-lg-n1, + .my-lg-n1 { + margin-bottom: -0.25rem !important; + } + .ml-lg-n1, + .mx-lg-n1 { + margin-left: -0.25rem !important; + } + .m-lg-n2 { + margin: -0.5rem !important; + } + .mt-lg-n2, + .my-lg-n2 { + margin-top: -0.5rem !important; + } + .mr-lg-n2, + .mx-lg-n2 { + margin-right: -0.5rem !important; + } + .mb-lg-n2, + .my-lg-n2 { + margin-bottom: -0.5rem !important; + } + .ml-lg-n2, + .mx-lg-n2 { + margin-left: -0.5rem !important; + } + .m-lg-n3 { + margin: -1rem !important; + } + .mt-lg-n3, + .my-lg-n3 { + margin-top: -1rem !important; + } + .mr-lg-n3, + .mx-lg-n3 { + margin-right: -1rem !important; + } + .mb-lg-n3, + .my-lg-n3 { + margin-bottom: -1rem !important; + } + .ml-lg-n3, + .mx-lg-n3 { + margin-left: -1rem !important; + } + .m-lg-n4 { + margin: -1.5rem !important; + } + .mt-lg-n4, + .my-lg-n4 { + margin-top: -1.5rem !important; + } + .mr-lg-n4, + .mx-lg-n4 { + margin-right: -1.5rem !important; + } + .mb-lg-n4, + .my-lg-n4 { + margin-bottom: -1.5rem !important; + } + .ml-lg-n4, + .mx-lg-n4 { + margin-left: -1.5rem !important; + } + .m-lg-n5 { + margin: -3rem !important; + } + .mt-lg-n5, + .my-lg-n5 { + margin-top: -3rem !important; + } + .mr-lg-n5, + .mx-lg-n5 { + margin-right: -3rem !important; + } + .mb-lg-n5, + .my-lg-n5 { + margin-bottom: -3rem !important; + } + .ml-lg-n5, + .mx-lg-n5 { + margin-left: -3rem !important; + } + .m-lg-auto { + margin: auto !important; + } + .mt-lg-auto, + .my-lg-auto { + margin-top: auto !important; + } + .mr-lg-auto, + .mx-lg-auto { + margin-right: auto !important; + } + .mb-lg-auto, + .my-lg-auto { + margin-bottom: auto !important; + } + .ml-lg-auto, + .mx-lg-auto { + margin-left: auto !important; + } +} + +@media (min-width: 1200px) { + .m-xl-0 { + margin: 0 !important; + } + .mt-xl-0, + .my-xl-0 { + margin-top: 0 !important; + } + .mr-xl-0, + .mx-xl-0 { + margin-right: 0 !important; + } + .mb-xl-0, + .my-xl-0 { + margin-bottom: 0 !important; + } + .ml-xl-0, + .mx-xl-0 { + margin-left: 0 !important; + } + .m-xl-1 { + margin: 0.25rem !important; + } + .mt-xl-1, + .my-xl-1 { + margin-top: 0.25rem !important; + } + .mr-xl-1, + .mx-xl-1 { + margin-right: 0.25rem !important; + } + .mb-xl-1, + .my-xl-1 { + margin-bottom: 0.25rem !important; + } + .ml-xl-1, + .mx-xl-1 { + margin-left: 0.25rem !important; + } + .m-xl-2 { + margin: 0.5rem !important; + } + .mt-xl-2, + .my-xl-2 { + margin-top: 0.5rem !important; + } + .mr-xl-2, + .mx-xl-2 { + margin-right: 0.5rem !important; + } + .mb-xl-2, + .my-xl-2 { + margin-bottom: 0.5rem !important; + } + .ml-xl-2, + .mx-xl-2 { + margin-left: 0.5rem !important; + } + .m-xl-3 { + margin: 1rem !important; + } + .mt-xl-3, + .my-xl-3 { + margin-top: 1rem !important; + } + .mr-xl-3, + .mx-xl-3 { + margin-right: 1rem !important; + } + .mb-xl-3, + .my-xl-3 { + margin-bottom: 1rem !important; + } + .ml-xl-3, + .mx-xl-3 { + margin-left: 1rem !important; + } + .m-xl-4 { + margin: 1.5rem !important; + } + .mt-xl-4, + .my-xl-4 { + margin-top: 1.5rem !important; + } + .mr-xl-4, + .mx-xl-4 { + margin-right: 1.5rem !important; + } + .mb-xl-4, + .my-xl-4 { + margin-bottom: 1.5rem !important; + } + .ml-xl-4, + .mx-xl-4 { + margin-left: 1.5rem !important; + } + .m-xl-5 { + margin: 3rem !important; + } + .mt-xl-5, + .my-xl-5 { + margin-top: 3rem !important; + } + .mr-xl-5, + .mx-xl-5 { + margin-right: 3rem !important; + } + .mb-xl-5, + .my-xl-5 { + margin-bottom: 3rem !important; + } + .ml-xl-5, + .mx-xl-5 { + margin-left: 3rem !important; + } + .p-xl-0 { + padding: 0 !important; + } + .pt-xl-0, + .py-xl-0 { + padding-top: 0 !important; + } + .pr-xl-0, + .px-xl-0 { + padding-right: 0 !important; + } + .pb-xl-0, + .py-xl-0 { + padding-bottom: 0 !important; + } + .pl-xl-0, + .px-xl-0 { + padding-left: 0 !important; + } + .p-xl-1 { + padding: 0.25rem !important; + } + .pt-xl-1, + .py-xl-1 { + padding-top: 0.25rem !important; + } + .pr-xl-1, + .px-xl-1 { + padding-right: 0.25rem !important; + } + .pb-xl-1, + .py-xl-1 { + padding-bottom: 0.25rem !important; + } + .pl-xl-1, + .px-xl-1 { + padding-left: 0.25rem !important; + } + .p-xl-2 { + padding: 0.5rem !important; + } + .pt-xl-2, + .py-xl-2 { + padding-top: 0.5rem !important; + } + .pr-xl-2, + .px-xl-2 { + padding-right: 0.5rem !important; + } + .pb-xl-2, + .py-xl-2 { + padding-bottom: 0.5rem !important; + } + .pl-xl-2, + .px-xl-2 { + padding-left: 0.5rem !important; + } + .p-xl-3 { + padding: 1rem !important; + } + .pt-xl-3, + .py-xl-3 { + padding-top: 1rem !important; + } + .pr-xl-3, + .px-xl-3 { + padding-right: 1rem !important; + } + .pb-xl-3, + .py-xl-3 { + padding-bottom: 1rem !important; + } + .pl-xl-3, + .px-xl-3 { + padding-left: 1rem !important; + } + .p-xl-4 { + padding: 1.5rem !important; + } + .pt-xl-4, + .py-xl-4 { + padding-top: 1.5rem !important; + } + .pr-xl-4, + .px-xl-4 { + padding-right: 1.5rem !important; + } + .pb-xl-4, + .py-xl-4 { + padding-bottom: 1.5rem !important; + } + .pl-xl-4, + .px-xl-4 { + padding-left: 1.5rem !important; + } + .p-xl-5 { + padding: 3rem !important; + } + .pt-xl-5, + .py-xl-5 { + padding-top: 3rem !important; + } + .pr-xl-5, + .px-xl-5 { + padding-right: 3rem !important; + } + .pb-xl-5, + .py-xl-5 { + padding-bottom: 3rem !important; + } + .pl-xl-5, + .px-xl-5 { + padding-left: 3rem !important; + } + .m-xl-n1 { + margin: -0.25rem !important; + } + .mt-xl-n1, + .my-xl-n1 { + margin-top: -0.25rem !important; + } + .mr-xl-n1, + .mx-xl-n1 { + margin-right: -0.25rem !important; + } + .mb-xl-n1, + .my-xl-n1 { + margin-bottom: -0.25rem !important; + } + .ml-xl-n1, + .mx-xl-n1 { + margin-left: -0.25rem !important; + } + .m-xl-n2 { + margin: -0.5rem !important; + } + .mt-xl-n2, + .my-xl-n2 { + margin-top: -0.5rem !important; + } + .mr-xl-n2, + .mx-xl-n2 { + margin-right: -0.5rem !important; + } + .mb-xl-n2, + .my-xl-n2 { + margin-bottom: -0.5rem !important; + } + .ml-xl-n2, + .mx-xl-n2 { + margin-left: -0.5rem !important; + } + .m-xl-n3 { + margin: -1rem !important; + } + .mt-xl-n3, + .my-xl-n3 { + margin-top: -1rem !important; + } + .mr-xl-n3, + .mx-xl-n3 { + margin-right: -1rem !important; + } + .mb-xl-n3, + .my-xl-n3 { + margin-bottom: -1rem !important; + } + .ml-xl-n3, + .mx-xl-n3 { + margin-left: -1rem !important; + } + .m-xl-n4 { + margin: -1.5rem !important; + } + .mt-xl-n4, + .my-xl-n4 { + margin-top: -1.5rem !important; + } + .mr-xl-n4, + .mx-xl-n4 { + margin-right: -1.5rem !important; + } + .mb-xl-n4, + .my-xl-n4 { + margin-bottom: -1.5rem !important; + } + .ml-xl-n4, + .mx-xl-n4 { + margin-left: -1.5rem !important; + } + .m-xl-n5 { + margin: -3rem !important; + } + .mt-xl-n5, + .my-xl-n5 { + margin-top: -3rem !important; + } + .mr-xl-n5, + .mx-xl-n5 { + margin-right: -3rem !important; + } + .mb-xl-n5, + .my-xl-n5 { + margin-bottom: -3rem !important; + } + .ml-xl-n5, + .mx-xl-n5 { + margin-left: -3rem !important; + } + .m-xl-auto { + margin: auto !important; + } + .mt-xl-auto, + .my-xl-auto { + margin-top: auto !important; + } + .mr-xl-auto, + .mx-xl-auto { + margin-right: auto !important; + } + .mb-xl-auto, + .my-xl-auto { + margin-bottom: auto !important; + } + .ml-xl-auto, + .mx-xl-auto { + margin-left: auto !important; + } +} + +.text-monospace { + font-family: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace !important; +} + +.text-justify { + text-align: justify !important; +} + +.text-wrap { + white-space: normal !important; +} + +.text-nowrap { + white-space: nowrap !important; +} + +.text-truncate { + overflow: hidden; + text-overflow: ellipsis; + white-space: nowrap; +} + +.text-left { + text-align: left !important; +} + +.text-right { + text-align: right !important; +} + +.text-center { + text-align: center !important; +} + +@media (min-width: 576px) { + .text-sm-left { + text-align: left !important; + } + .text-sm-right { + text-align: right !important; + } + .text-sm-center { + text-align: center !important; + } +} + +@media (min-width: 768px) { + .text-md-left { + text-align: left !important; + } + .text-md-right { + text-align: right !important; + } + .text-md-center { + text-align: center !important; + } +} + +@media (min-width: 992px) { + .text-lg-left { + text-align: left !important; + } + .text-lg-right { + text-align: right !important; + } + .text-lg-center { + text-align: center !important; + } +} + +@media (min-width: 1200px) { + .text-xl-left { + text-align: left !important; + } + .text-xl-right { + text-align: right !important; + } + .text-xl-center { + text-align: center !important; + } +} + +.text-lowercase { + text-transform: lowercase !important; +} + +.text-uppercase { + text-transform: uppercase !important; +} + +.text-capitalize { + text-transform: capitalize !important; +} + +.font-weight-light { + font-weight: 300 !important; +} + +.font-weight-lighter { + font-weight: lighter !important; +} + +.font-weight-normal { + font-weight: 400 !important; +} + +.font-weight-bold { + font-weight: 700 !important; +} + +.font-weight-bolder { + font-weight: bolder !important; +} + +.font-italic { + font-style: italic !important; +} + +.text-white { + color: #fff !important; +} + +.text-primary { + color: #007bff !important; +} + +a.text-primary:hover, a.text-primary:focus { + color: #0056b3 !important; +} + +.text-secondary { + color: #6c757d !important; +} + +a.text-secondary:hover, a.text-secondary:focus { + color: #494f54 !important; +} + +.text-success { + color: #28a745 !important; +} + +a.text-success:hover, a.text-success:focus { + color: #19692c !important; +} + +.text-info { + color: #17a2b8 !important; +} + +a.text-info:hover, a.text-info:focus { + color: #0f6674 !important; +} + +.text-warning { + color: #ffc107 !important; +} + +a.text-warning:hover, a.text-warning:focus { + color: #ba8b00 !important; +} + +.text-danger { + color: #dc3545 !important; +} + +a.text-danger:hover, a.text-danger:focus { + color: #a71d2a !important; +} + +.text-light { + color: #f8f9fa !important; +} + +a.text-light:hover, a.text-light:focus { + color: #cbd3da !important; +} + +.text-dark { + color: #343a40 !important; +} + +a.text-dark:hover, a.text-dark:focus { + color: #121416 !important; +} + +.text-body { + color: #212529 !important; +} + +.text-muted { + color: #6c757d !important; +} + +.text-black-50 { + color: rgba(0, 0, 0, 0.5) !important; +} + +.text-white-50 { + color: rgba(255, 255, 255, 0.5) !important; +} + +.text-hide { + font: 0/0 a; + color: transparent; + text-shadow: none; + background-color: transparent; + border: 0; +} + +.text-decoration-none { + text-decoration: none !important; +} + +.text-break { + word-break: break-word !important; + overflow-wrap: break-word !important; +} + +.text-reset { + color: inherit !important; +} + +.visible { + visibility: visible !important; +} + +.invisible { + visibility: hidden !important; +} + +@media print { + *, + *::before, + *::after { + text-shadow: none !important; + box-shadow: none !important; + } + a:not(.btn) { + text-decoration: underline; + } + abbr[title]::after { + content: " (" attr(title) ")"; + } + pre { + white-space: pre-wrap !important; + } + pre, + blockquote { + border: 1px solid #adb5bd; + page-break-inside: avoid; + } + thead { + display: table-header-group; + } + tr, + img { + page-break-inside: avoid; + } + p, + h2, + h3 { + orphans: 3; + widows: 3; + } + h2, + h3 { + page-break-after: avoid; + } + @page { + size: a3; + } + body { + min-width: 992px !important; + } + .container { + min-width: 992px !important; + } + .navbar { + display: none; + } + .badge { + border: 1px solid #000; + } + .table { + border-collapse: collapse !important; + } + .table td, + .table th { + background-color: #fff !important; + } + .table-bordered th, + .table-bordered td { + border: 1px solid #dee2e6 !important; + } + .table-dark { + color: inherit; + } + .table-dark th, + .table-dark td, + .table-dark thead th, + .table-dark tbody + tbody { + border-color: #dee2e6; + } + .table .thead-dark th { + color: inherit; + border-color: #dee2e6; + } +} +/*# sourceMappingURL=bootstrap.css.map */ \ No newline at end of file diff --git a/website/css/docs.css b/website/css/docs.css new file mode 100644 index 00000000000..1f4bbbbba16 --- /dev/null +++ b/website/css/docs.css @@ -0,0 +1,186 @@ +#sidebar { + position: fixed; + z-index: 50; + left: 0; +} + +#toc.toc-right { + position: fixed; +} + +#languages-dropdown .disabled { + color: #666 !important; +} + +.toc-right { + right: 0; +} + +#sidebar-nav .nav-link.active { + font-weight: bold; +} + +.headerlink { + display: none; + text-decoration: none !important; + margin-left: .5rem; +} + +#content { + overflow-x: scroll; +} + +#content table { + border: 1px solid #dee2e6; + width: 100%; + margin-bottom: 1rem; + overflow-x: auto; +} + +#content thead { + background: #444451; + color: #fff; +} + +#content td, #content th { + padding: .75rem; + vertical-align: top; +} + +#content td { + border: 1px solid #dee2e6; +} + +#content th { + border: 1px solid #444451; +} + +#content code { + color: #111; + background: #eee; + padding: 2px; +} + + +#content pre { + background: #eee; + padding: 1rem; +} + +#docsearch-input:focus, #docsearch-input:active { + border: 0; + color: #efefef!important; +} + +@media (max-width: 768px) { + #search-form { + width: 50%; + } + .algolia-autocomplete, #docsearch-input, #search-form .input-group { + width: 100%; + } +} + +.algolia-autocomplete .algolia-docsearch-suggestion--subcategory-column { + display: none !important; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--content { + float: none !important; + width: 100% !important; + background-color: #444451; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--content:before { + content: none !important; +} + +.algolia-autocomplete .ds-dropdown-menu { + max-height: 512px; + overflow-x: hidden; + overflow-y: auto; +} + +.algolia-autocomplete .ds-dropdown-menu, +.algolia-autocomplete .ds-dropdown-menu [class^=ds-dataset-], +.algolia-autocomplete .algolia-docsearch-suggestion, +.algolia-autocomplete .ds-dropdown-menu:before { + background: #1c1c1c !important; + border-color: #333; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--category-header, +.algolia-autocomplete .algolia-docsearch-suggestion--subcategory-column, +.algolia-autocomplete .algolia-docsearch-suggestion--title, +.algolia-autocomplete .algolia-docsearch-suggestion--text { + color: #efefef; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--highlight { + color: #f14600; +} + +#toc .nav-link { + color: #333; +} + +.toc-left, #toc.toc-left .nav-link { + color: #efefef; +} + +#toc .nav { + flex-wrap: nowrap !important; +} + +.toc-muted { + color: #888 !important; +} + +#toc .nav-link:hover { + color: #f14600 !important; +} + +@media print { + #content-wrapper { + -ms-flex: 0 0 100%; + flex: 0 0 100%; + max-width: 100%; + } +} + +@media (prefers-color-scheme: dark) { + body[data-spy] { + background: #1c1c1c; + color: #efefef; + } + + #sidebar { + background: #333; + } + + #content table { + border: 1px solid #2a2b2c; + } + + #content thead { + background: #333; + } + + #content td { + border: 1px solid #444451; + } + #content code { + background: #444; + color: #eee; + } + + #content pre { + background: #444; + color: #eee; + } + + #toc .nav-link { + color: #bbb; + } +} + diff --git a/website/css/docsearch.css b/website/css/docsearch.css new file mode 100644 index 00000000000..dc3c442d356 --- /dev/null +++ b/website/css/docsearch.css @@ -0,0 +1,556 @@ +.searchbox { + display: inline-block; + position: relative; + width: 200px; + height: 32px !important; + white-space: nowrap; + box-sizing: border-box; + visibility: visible !important; +} + +.searchbox .algolia-autocomplete { + display: block; + width: 100%; + height: 100%; +} + +.searchbox__wrapper { + width: 100%; + height: 100%; + z-index: 999; + position: relative; +} + +.searchbox__input { + display: inline-block; + box-sizing: border-box; + transition: box-shadow 0.4s ease, background 0.4s ease; + border: 0; + border-radius: 16px; + box-shadow: inset 0 0 0 1px #cccccc; + background: #ffffff !important; + padding: 0; + padding-right: 26px; + padding-left: 32px; + width: 100%; + height: 100%; + vertical-align: middle; + white-space: normal; + font-size: 12px; + -webkit-appearance: none; + -moz-appearance: none; + appearance: none; +} + +.searchbox__input::-webkit-search-decoration, .searchbox__input::-webkit-search-cancel-button, .searchbox__input::-webkit-search-results-button, .searchbox__input::-webkit-search-results-decoration { + display: none; +} + +.searchbox__input:hover { + box-shadow: inset 0 0 0 1px #b3b3b3; +} + +.searchbox__input:focus, .searchbox__input:active { + outline: 0; + box-shadow: inset 0 0 0 1px #aaaaaa; + background: #ffffff; +} + +.searchbox__input::-webkit-input-placeholder { + color: #aaaaaa; +} + +.searchbox__input:-ms-input-placeholder { + color: #aaaaaa; +} + +.searchbox__input::-ms-input-placeholder { + color: #aaaaaa; +} + +.searchbox__input::placeholder { + color: #aaaaaa; +} + +.searchbox__submit { + position: absolute; + top: 0; + margin: 0; + border: 0; + border-radius: 16px 0 0 16px; + background-color: rgba(69, 142, 225, 0); + padding: 0; + width: 32px; + height: 100%; + vertical-align: middle; + text-align: center; + font-size: inherit; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + right: inherit; + left: 0; +} + +.searchbox__submit::before { + display: inline-block; + margin-right: -4px; + height: 100%; + vertical-align: middle; + content: ''; +} + +.searchbox__submit:hover, .searchbox__submit:active { + cursor: pointer; +} + +.searchbox__submit:focus { + outline: 0; +} + +.searchbox__submit svg { + width: 14px; + height: 14px; + vertical-align: middle; + fill: #6d7e96; +} + +.searchbox__reset { + display: block; + position: absolute; + top: 8px; + right: 8px; + margin: 0; + border: 0; + background: none; + cursor: pointer; + padding: 0; + font-size: inherit; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + fill: rgba(0, 0, 0, 0.5); +} + +.searchbox__reset.hide { + display: none; +} + +.searchbox__reset:focus { + outline: 0; +} + +.searchbox__reset svg { + display: block; + margin: 4px; + width: 8px; + height: 8px; +} + +.searchbox__input:valid ~ .searchbox__reset { + display: block; + -webkit-animation-name: sbx-reset-in; + animation-name: sbx-reset-in; + -webkit-animation-duration: 0.15s; + animation-duration: 0.15s; +} + +@-webkit-keyframes sbx-reset-in { + 0% { + -webkit-transform: translate3d(-20%, 0, 0); + transform: translate3d(-20%, 0, 0); + opacity: 0; + } + 100% { + -webkit-transform: none; + transform: none; + opacity: 1; + } +} + +@keyframes sbx-reset-in { + 0% { + -webkit-transform: translate3d(-20%, 0, 0); + transform: translate3d(-20%, 0, 0); + opacity: 0; + } + 100% { + -webkit-transform: none; + transform: none; + opacity: 1; + } +} + +.algolia-autocomplete.algolia-autocomplete-right .ds-dropdown-menu { + right: 0 !important; + left: inherit !important; +} + +.algolia-autocomplete.algolia-autocomplete-right .ds-dropdown-menu:before { + right: 48px; +} + +.algolia-autocomplete.algolia-autocomplete-left .ds-dropdown-menu { + left: 0 !important; + right: inherit !important; +} + +.algolia-autocomplete.algolia-autocomplete-left .ds-dropdown-menu:before { + left: 48px; +} + +.algolia-autocomplete .ds-dropdown-menu { + position: relative; + top: -6px; + border-radius: 4px; + margin: 6px 0 0; + padding: 0; + text-align: left; + height: auto; + position: relative; + background: transparent; + border: none; + z-index: 999; + max-width: 600px; + min-width: 500px; + box-shadow: 0 1px 0 0 rgba(0, 0, 0, 0.2), 0 2px 3px 0 rgba(0, 0, 0, 0.1); +} + +.algolia-autocomplete .ds-dropdown-menu:before { + display: block; + position: absolute; + content: ''; + width: 14px; + height: 14px; + background: #fff; + z-index: 1000; + top: -7px; + border-top: 1px solid #d9d9d9; + border-right: 1px solid #d9d9d9; + -webkit-transform: rotate(-45deg); + transform: rotate(-45deg); + border-radius: 2px; +} + +.algolia-autocomplete .ds-dropdown-menu .ds-suggestions { + position: relative; + z-index: 1000; + margin-top: 8px; +} + +.algolia-autocomplete .ds-dropdown-menu .ds-suggestions a:hover { + text-decoration: none; +} + +.algolia-autocomplete .ds-dropdown-menu .ds-suggestion { + cursor: pointer; +} + +.algolia-autocomplete .ds-dropdown-menu .ds-suggestion.ds-cursor .algolia-docsearch-suggestion.suggestion-layout-simple { + background-color: rgba(69, 142, 225, 0.05); +} + +.algolia-autocomplete .ds-dropdown-menu .ds-suggestion.ds-cursor .algolia-docsearch-suggestion:not(.suggestion-layout-simple) .algolia-docsearch-suggestion--content { + background-color: rgba(69, 142, 225, 0.05); +} + +.algolia-autocomplete .ds-dropdown-menu [class^='ds-dataset-'] { + position: relative; + border: solid 1px #d9d9d9; + background: #fff; + border-radius: 4px; + overflow: auto; + padding: 0 8px 8px; +} + +.algolia-autocomplete .ds-dropdown-menu * { + box-sizing: border-box; +} + +.algolia-autocomplete .algolia-docsearch-suggestion { + display: block; + position: relative; + padding: 0 8px; + background: #fff; + color: #02060c; + overflow: hidden; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--highlight { + color: #174d8c; + background: rgba(143, 187, 237, 0.1); + padding: 0.1em 0.05em; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--category-header .algolia-docsearch-suggestion--category-header-lvl0 +.algolia-docsearch-suggestion--highlight, +.algolia-autocomplete .algolia-docsearch-suggestion--category-header .algolia-docsearch-suggestion--category-header-lvl1 +.algolia-docsearch-suggestion--highlight { + padding: 0 0 1px; + background: inherit; + box-shadow: inset 0 -2px 0 0 rgba(69, 142, 225, 0.8); + color: inherit; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--text .algolia-docsearch-suggestion--highlight { + padding: 0 0 1px; + background: inherit; + box-shadow: inset 0 -2px 0 0 rgba(69, 142, 225, 0.8); + color: inherit; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--content { + display: block; + float: right; + width: 70%; + position: relative; + padding: 5.33333px 0 5.33333px 10.66667px; + cursor: pointer; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--content:before { + content: ''; + position: absolute; + display: block; + top: 0; + height: 100%; + width: 1px; + background: #ddd; + left: -1px; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--category-header { + position: relative; + border-bottom: 1px solid #ddd; + display: none; + margin-top: 8px; + padding: 4px 0; + font-size: 1em; + color: #33363d; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--wrapper { + width: 100%; + float: left; + padding: 8px 0 0 0; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--subcategory-column { + float: left; + width: 30%; + padding-left: 0; + text-align: right; + position: relative; + padding: 5.33333px 10.66667px; + color: #a4a7ae; + font-size: 0.9em; + word-wrap: break-word; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--subcategory-column:before { + content: ''; + position: absolute; + display: block; + top: 0; + height: 100%; + width: 1px; + background: #ddd; + right: 0; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--subcategory-inline { + display: none; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--title { + margin-bottom: 4px; + color: #02060c; + font-size: 0.9em; + font-weight: bold; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--text { + display: block; + line-height: 1.2em; + font-size: 0.85em; + color: #63676d; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--no-results { + width: 100%; + padding: 8px 0; + text-align: center; + font-size: 1.2em; +} + +.algolia-autocomplete .algolia-docsearch-suggestion--no-results::before { + display: none; +} + +.algolia-autocomplete .algolia-docsearch-suggestion code { + padding: 1px 5px; + font-size: 90%; + border: none; + color: #222222; + background-color: #ebebeb; + border-radius: 3px; + font-family: Menlo, Monaco, Consolas, 'Courier New', monospace; +} + +.algolia-autocomplete .algolia-docsearch-suggestion code .algolia-docsearch-suggestion--highlight { + background: none; +} + +.algolia-autocomplete .algolia-docsearch-suggestion.algolia-docsearch-suggestion__main .algolia-docsearch-suggestion--category-header { + display: block; +} + +.algolia-autocomplete .algolia-docsearch-suggestion.algolia-docsearch-suggestion__secondary { + display: block; +} + +@media all and (min-width: 768px) { + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--subcategory-column { + display: block; + } +} + +@media all and (max-width: 768px) { + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--subcategory-column { + display: inline-block; + width: auto; + text-align: left; + float: left; + padding: 0; + color: #02060c; + font-size: 0.9em; + font-weight: bold; + text-align: left; + opacity: 0.5; + } + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--subcategory-column:before { + display: none; + } + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--subcategory-column:after { + content: '|'; + } + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--content { + display: inline-block; + width: auto; + text-align: left; + float: left; + padding: 0; + } + .algolia-autocomplete .algolia-docsearch-suggestion .algolia-docsearch-suggestion--content:before { + display: none; + } +} + +.algolia-autocomplete .suggestion-layout-simple.algolia-docsearch-suggestion { + border-bottom: solid 1px #eee; + padding: 8px; + margin: 0; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--content { + width: 100%; + padding: 0; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--content::before { + display: none; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--category-header { + margin: 0; + padding: 0; + display: block; + width: 100%; + border: none; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--category-header-lvl0 { + opacity: 0.6; + font-size: 0.85em; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--category-header-lvl1 { + opacity: 0.6; + font-size: 0.85em; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--category-header-lvl1::before { + background-image: url('data:image/svg+xml;utf8,'); + content: ''; + width: 10px; + height: 10px; + display: inline-block; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--wrapper { + width: 100%; + float: left; + margin: 0; + padding: 0; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--duplicate-content, .algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--subcategory-inline { + display: none !important; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--title { + margin: 0; + color: #458ee1; + font-size: 0.9em; + font-weight: normal; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--title::before { + content: '#'; + font-weight: bold; + color: #458ee1; + display: inline-block; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--text { + margin: 4px 0 0; + display: block; + line-height: 1.4em; + padding: 5.33333px 8px; + background: #f8f8f8; + font-size: 0.85em; + opacity: 0.8; +} + +.algolia-autocomplete .suggestion-layout-simple .algolia-docsearch-suggestion--text .algolia-docsearch-suggestion--highlight { + color: #3f4145; + font-weight: bold; + box-shadow: none; +} + +.algolia-autocomplete .algolia-docsearch-footer { + width: 134px; + height: 20px; + z-index: 2000; + margin-top: 10.66667px; + float: right; + font-size: 0; + line-height: 0; +} + +.algolia-autocomplete .algolia-docsearch-footer--logo { + background-image: url("data:image/svg+xml,%3Csvg width='168' height='24' xmlns='http://www.w3.org/2000/svg'%3E%3Cg fill='none' fill-rule='evenodd'%3E%3Cpath d='M78.988.938h16.594a2.968 2.968 0 0 1 2.966 2.966V20.5a2.967 2.967 0 0 1-2.966 2.964H78.988a2.967 2.967 0 0 1-2.966-2.964V3.897A2.961 2.961 0 0 1 78.988.938zm41.937 17.866c-4.386.02-4.386-3.54-4.386-4.106l-.007-13.336 2.675-.424v13.254c0 .322 0 2.358 1.718 2.364v2.248zm-10.846-2.18c.821 0 1.43-.047 1.855-.129v-2.719a6.334 6.334 0 0 0-1.574-.199c-.295 0-.596.021-.897.069a2.699 2.699 0 0 0-.814.24c-.24.116-.439.28-.582.491-.15.212-.219.335-.219.656 0 .628.219.991.616 1.23s.938.362 1.615.362zm-.233-9.7c.883 0 1.629.109 2.231.328.602.218 1.088.525 1.444.915.363.396.609.922.76 1.483.157.56.232 1.175.232 1.85v6.874c-.41.089-1.034.19-1.868.314-.834.123-1.772.185-2.813.185-.69 0-1.327-.069-1.895-.198a4.001 4.001 0 0 1-1.471-.636 3.085 3.085 0 0 1-.951-1.134c-.226-.465-.343-1.12-.343-1.803 0-.656.13-1.073.384-1.525.26-.45.608-.819 1.047-1.106.445-.287.95-.492 1.532-.615a8.8 8.8 0 0 1 1.82-.185 8.404 8.404 0 0 1 1.972.24v-.438c0-.307-.035-.6-.11-.874a1.88 1.88 0 0 0-.384-.73 1.784 1.784 0 0 0-.724-.493 3.164 3.164 0 0 0-1.143-.205c-.616 0-1.177.075-1.69.164a7.735 7.735 0 0 0-1.26.307l-.321-2.192c.335-.117.834-.233 1.478-.349a10.98 10.98 0 0 1 2.073-.178zm52.842 9.626c.822 0 1.43-.048 1.854-.13V13.7a6.347 6.347 0 0 0-1.574-.199c-.294 0-.595.021-.896.069a2.7 2.7 0 0 0-.814.24 1.46 1.46 0 0 0-.582.491c-.15.212-.218.335-.218.656 0 .628.218.991.615 1.23.404.245.938.362 1.615.362zm-.226-9.694c.883 0 1.629.108 2.231.327.602.219 1.088.526 1.444.915.355.39.609.923.759 1.483.158.56.233 1.175.233 1.852v6.873c-.41.088-1.034.19-1.868.314-.834.123-1.772.184-2.813.184-.69 0-1.327-.068-1.895-.198a4.001 4.001 0 0 1-1.471-.635 3.085 3.085 0 0 1-.951-1.134c-.226-.465-.343-1.12-.343-1.804 0-.656.13-1.073.384-1.524.26-.45.608-.82 1.047-1.107.445-.286.95-.491 1.532-.614a8.803 8.803 0 0 1 2.751-.13c.329.034.671.096 1.04.185v-.437a3.3 3.3 0 0 0-.109-.875 1.873 1.873 0 0 0-.384-.731 1.784 1.784 0 0 0-.724-.492 3.165 3.165 0 0 0-1.143-.205c-.616 0-1.177.075-1.69.164-.514.089-.938.191-1.26.307l-.321-2.193c.335-.116.834-.232 1.478-.348a11.633 11.633 0 0 1 2.073-.177zm-8.034-1.271a1.626 1.626 0 0 1-1.628-1.62c0-.895.725-1.62 1.628-1.62.904 0 1.63.725 1.63 1.62 0 .895-.733 1.62-1.63 1.62zm1.348 13.22h-2.689V7.27l2.69-.423v11.956zm-4.714 0c-4.386.02-4.386-3.54-4.386-4.107l-.008-13.336 2.676-.424v13.254c0 .322 0 2.358 1.718 2.364v2.248zm-8.698-5.903c0-1.156-.253-2.119-.746-2.788-.493-.677-1.183-1.01-2.067-1.01-.882 0-1.574.333-2.065 1.01-.493.676-.733 1.632-.733 2.788 0 1.168.246 1.953.74 2.63.492.683 1.183 1.018 2.066 1.018.882 0 1.574-.342 2.067-1.019.492-.683.738-1.46.738-2.63zm2.737-.007c0 .902-.13 1.584-.397 2.33a5.52 5.52 0 0 1-1.128 1.906 4.986 4.986 0 0 1-1.752 1.223c-.685.286-1.739.45-2.265.45-.528-.006-1.574-.157-2.252-.45a5.096 5.096 0 0 1-1.744-1.223c-.487-.527-.863-1.162-1.137-1.906a6.345 6.345 0 0 1-.41-2.33c0-.902.123-1.77.397-2.508a5.554 5.554 0 0 1 1.15-1.892 5.133 5.133 0 0 1 1.75-1.216c.679-.287 1.425-.423 2.232-.423.808 0 1.553.142 2.237.423.685.286 1.274.69 1.753 1.216a5.644 5.644 0 0 1 1.135 1.892c.287.738.431 1.606.431 2.508zm-20.138 0c0 1.12.246 2.363.738 2.882.493.52 1.13.78 1.91.78.424 0 .828-.062 1.204-.178.377-.116.677-.253.917-.417V9.33a10.476 10.476 0 0 0-1.766-.226c-.971-.028-1.71.37-2.23 1.004-.513.636-.773 1.75-.773 2.788zm7.438 5.274c0 1.824-.466 3.156-1.404 4.004-.936.846-2.367 1.27-4.296 1.27-.705 0-2.17-.137-3.34-.396l.431-2.118c.98.205 2.272.26 2.95.26 1.074 0 1.84-.219 2.299-.656.459-.437.684-1.086.684-1.948v-.437a8.07 8.07 0 0 1-1.047.397c-.43.13-.93.198-1.492.198-.739 0-1.41-.116-2.018-.349a4.206 4.206 0 0 1-1.567-1.025c-.431-.45-.774-1.017-1.013-1.694-.24-.677-.363-1.885-.363-2.773 0-.834.13-1.88.384-2.577.26-.696.629-1.298 1.129-1.796.493-.498 1.095-.881 1.8-1.162a6.605 6.605 0 0 1 2.428-.457c.87 0 1.67.109 2.45.24.78.129 1.444.265 1.985.415V18.17z' fill='%235468FF'/%3E%3Cpath d='M6.972 6.677v1.627c-.712-.446-1.52-.67-2.425-.67-.585 0-1.045.13-1.38.391a1.24 1.24 0 0 0-.502 1.03c0 .425.164.765.494 1.02.33.256.835.532 1.516.83.447.192.795.356 1.045.495.25.138.537.332.862.582.324.25.563.548.718.894.154.345.23.741.23 1.188 0 .947-.334 1.691-1.004 2.234-.67.542-1.537.814-2.601.814-1.18 0-2.16-.229-2.936-.686v-1.708c.84.628 1.814.942 2.92.942.585 0 1.048-.136 1.388-.407.34-.271.51-.646.51-1.125 0-.287-.1-.55-.302-.79-.203-.24-.42-.42-.655-.542-.234-.123-.585-.29-1.053-.503-.276-.127-.47-.218-.582-.271a13.67 13.67 0 0 1-.55-.287 4.275 4.275 0 0 1-.567-.351 6.92 6.92 0 0 1-.455-.4c-.18-.17-.31-.34-.39-.51-.08-.17-.155-.37-.224-.598a2.553 2.553 0 0 1-.104-.742c0-.915.333-1.638.998-2.17.664-.532 1.523-.798 2.576-.798.968 0 1.793.17 2.473.51zm7.468 5.696v-.287c-.022-.607-.187-1.088-.495-1.444-.309-.357-.75-.535-1.324-.535-.532 0-.99.194-1.373.583-.382.388-.622.949-.717 1.683h3.909zm1.005 2.792v1.404c-.596.34-1.383.51-2.362.51-1.255 0-2.255-.377-3-1.132-.744-.755-1.116-1.744-1.116-2.968 0-1.297.34-2.316 1.021-3.055.68-.74 1.548-1.11 2.6-1.11 1.033 0 1.852.323 2.458.966.606.644.91 1.572.91 2.784 0 .33-.033.676-.096 1.038h-5.314c.107.702.405 1.239.894 1.611.49.372 1.106.558 1.85.558.862 0 1.58-.202 2.155-.606zm6.605-1.77h-1.212c-.596 0-1.045.116-1.349.35-.303.234-.454.532-.454.894 0 .372.117.664.35.877.235.213.575.32 1.022.32.51 0 .912-.142 1.204-.424.293-.281.44-.651.44-1.108v-.91zm-4.068-2.554V9.325c.627-.361 1.457-.542 2.489-.542 2.116 0 3.175 1.026 3.175 3.08V17h-1.548v-.957c-.415.68-1.143 1.02-2.186 1.02-.766 0-1.38-.22-1.843-.661-.462-.442-.694-1.003-.694-1.684 0-.776.293-1.38.878-1.81.585-.431 1.404-.647 2.457-.647h1.34V11.8c0-.554-.133-.971-.399-1.253-.266-.282-.707-.423-1.324-.423a4.07 4.07 0 0 0-2.345.718zm9.333-1.93v1.42c.394-1 1.101-1.5 2.123-1.5.148 0 .313.016.494.048v1.531a1.885 1.885 0 0 0-.75-.143c-.542 0-.989.24-1.34.718-.351.479-.527 1.048-.527 1.707V17h-1.563V8.91h1.563zm5.01 4.084c.022.82.272 1.492.75 2.019.479.526 1.15.79 2.01.79.639 0 1.235-.176 1.788-.527v1.404c-.521.319-1.186.479-1.995.479-1.265 0-2.276-.4-3.031-1.197-.755-.798-1.133-1.792-1.133-2.984 0-1.16.38-2.151 1.14-2.975.761-.825 1.79-1.237 3.088-1.237.702 0 1.346.149 1.93.447v1.436a3.242 3.242 0 0 0-1.77-.495c-.84 0-1.513.266-2.019.798-.505.532-.758 1.213-.758 2.042zM40.24 5.72v4.579c.458-1 1.293-1.5 2.505-1.5.787 0 1.42.245 1.899.734.479.49.718 1.17.718 2.042V17h-1.564v-5.106c0-.553-.14-.98-.422-1.284-.282-.303-.652-.455-1.11-.455-.531 0-1.002.202-1.411.606-.41.405-.615 1.022-.615 1.851V17h-1.563V5.72h1.563zm14.966 10.02c.596 0 1.096-.253 1.5-.758.404-.506.606-1.157.606-1.955 0-.915-.202-1.62-.606-2.114-.404-.495-.92-.742-1.548-.742-.553 0-1.05.224-1.491.67-.442.447-.662 1.133-.662 2.058 0 .958.212 1.67.638 2.138.425.469.946.703 1.563.703zM53.004 5.72v4.42c.574-.894 1.388-1.341 2.44-1.341 1.022 0 1.857.383 2.506 1.149.649.766.973 1.781.973 3.047 0 1.138-.309 2.109-.925 2.912-.617.803-1.463 1.205-2.537 1.205-1.075 0-1.894-.447-2.457-1.34V17h-1.58V5.72h1.58zm9.908 11.104l-3.223-7.913h1.739l1.005 2.632 1.26 3.415c.096-.32.48-1.458 1.15-3.415l.909-2.632h1.66l-2.92 7.866c-.777 2.074-1.963 3.11-3.559 3.11a2.92 2.92 0 0 1-.734-.079v-1.34c.17.042.351.064.543.064 1.032 0 1.755-.57 2.17-1.708z' fill='%235D6494'/%3E%3Cpath d='M89.632 5.967v-.772a.978.978 0 0 0-.978-.977h-2.28a.978.978 0 0 0-.978.977v.793c0 .088.082.15.171.13a7.127 7.127 0 0 1 1.984-.28c.65 0 1.295.088 1.917.259.082.02.164-.04.164-.13m-6.248 1.01l-.39-.389a.977.977 0 0 0-1.382 0l-.465.465a.973.973 0 0 0 0 1.38l.383.383c.062.061.15.047.205-.014.226-.307.472-.601.746-.874.281-.28.568-.526.883-.751.068-.042.075-.137.02-.2m4.16 2.453v3.341c0 .096.104.165.192.117l2.97-1.537c.068-.034.089-.117.055-.184a3.695 3.695 0 0 0-3.08-1.866c-.068 0-.136.054-.136.13m0 8.048a4.489 4.489 0 0 1-4.49-4.482 4.488 4.488 0 0 1 4.49-4.482 4.488 4.488 0 0 1 4.489 4.482 4.484 4.484 0 0 1-4.49 4.482m0-10.85a6.363 6.363 0 1 0 0 12.729c3.518 0 6.372-2.85 6.372-6.368a6.358 6.358 0 0 0-6.371-6.36' fill='%23FFF'/%3E%3C/g%3E%3C/svg%3E%0A"); + background-repeat: no-repeat; + background-position: center; + background-size: 100%; + overflow: hidden; + text-indent: -9000px; + padding: 0 !important; + width: 100%; + height: 100%; + display: block; +} + +/*# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImRvY3NlYXJjaC5jc3MiXSwibmFtZXMiOltdLCJtYXBwaW5ncyI6IkFBQUE7RUFDRSxzQkFBc0I7RUFDdEIsbUJBQW1CO0VBQ25CLGFBQWE7RUFDYix3QkFBd0I7RUFDeEIsb0JBQW9CO0VBQ3BCLHVCQUF1QjtFQUN2QiwrQkFBK0I7Q0FDaEM7O0FBRUQ7RUFDRSxlQUFlO0VBQ2YsWUFBWTtFQUNaLGFBQWE7Q0FDZDs7QUFFRDtFQUNFLFlBQVk7RUFDWixhQUFhO0VBQ2IsYUFBYTtFQUNiLG1CQUFtQjtDQUNwQjs7QUFFRDtFQUNFLHNCQUFzQjtFQUN0Qix1QkFBdUI7RUFDdkIsdURBQXVEO0VBQ3ZELFVBQVU7RUFDVixvQkFBb0I7RUFDcEIsb0NBQW9DO0VBQ3BDLCtCQUErQjtFQUMvQixXQUFXO0VBQ1gsb0JBQW9CO0VBQ3BCLG1CQUFtQjtFQUNuQixZQUFZO0VBQ1osYUFBYTtFQUNiLHVCQUF1QjtFQUN2QixvQkFBb0I7RUFDcEIsZ0JBQWdCO0VBQ2hCLHlCQUFpQjtLQUFqQixzQkFBaUI7VUFBakIsaUJBQWlCO0NBQ2xCOztBQUVEO0VBQ0UsY0FBYztDQUNmOztBQUVEO0VBQ0Usb0NBQW9DO0NBQ3JDOztBQUVEO0VBQ0UsV0FBVztFQUNYLG9DQUFvQztFQUNwQyxvQkFBb0I7Q0FDckI7O0FBRUQ7RUFDRSxlQUFlO0NBQ2hCOztBQUZEO0VBQ0UsZUFBZTtDQUNoQjs7QUFGRDtFQUNFLGVBQWU7Q0FDaEI7O0FBRkQ7RUFDRSxlQUFlO0NBQ2hCOztBQUVEO0VBQ0UsbUJBQW1CO0VBQ25CLE9BQU87RUFDUCxVQUFVO0VBQ1YsVUFBVTtFQUNWLDZCQUE2QjtFQUM3Qix3Q0FBd0M7RUFDeEMsV0FBVztFQUNYLFlBQVk7RUFDWixhQUFhO0VBQ2IsdUJBQXVCO0VBQ3ZCLG1CQUFtQjtFQUNuQixtQkFBbUI7RUFDbkIsMEJBQWtCO0tBQWxCLHVCQUFrQjtNQUFsQixzQkFBa0I7VUFBbEIsa0JBQWtCO0VBQ2xCLGVBQWU7RUFDZixRQUFRO0NBQ1Q7O0FBRUQ7RUFDRSxzQkFBc0I7RUFDdEIsbUJBQW1CO0VBQ25CLGFBQWE7RUFDYix1QkFBdUI7RUFDdkIsWUFBWTtDQUNiOztBQUVEO0VBQ0UsZ0JBQWdCO0NBQ2pCOztBQUVEO0VBQ0UsV0FBVztDQUNaOztBQUVEO0VBQ0UsWUFBWTtFQUNaLGFBQWE7RUFDYix1QkFBdUI7RUFDdkIsY0FBYztDQUNmOztBQUVEO0VBQ0UsZUFBZTtFQUNmLG1CQUFtQjtFQUNuQixTQUFTO0VBQ1QsV0FBVztFQUNYLFVBQVU7RUFDVixVQUFVO0VBQ1YsaUJBQWlCO0VBQ2pCLGdCQUFnQjtFQUNoQixXQUFXO0VBQ1gsbUJBQW1CO0VBQ25CLDBCQUFrQjtLQUFsQix1QkFBa0I7TUFBbEIsc0JBQWtCO1VBQWxCLGtCQUFrQjtFQUNsQix5QkFBeUI7Q0FDMUI7O0FBRUQ7RUFDRSxjQUFjO0NBQ2Y7O0FBRUQ7RUFDRSxXQUFXO0NBQ1o7O0FBRUQ7RUFDRSxlQUFlO0VBQ2YsWUFBWTtFQUNaLFdBQVc7RUFDWCxZQUFZO0NBQ2I7O0FBRUQ7RUFDRSxlQUFlO0VBQ2YscUNBQTZCO1VBQTdCLDZCQUE2QjtFQUM3QixrQ0FBMEI7VUFBMUIsMEJBQTBCO0NBQzNCOztBQUVEO0VBQ0U7SUFDRSwyQ0FBbUM7WUFBbkMsbUNBQW1DO0lBQ25DLFdBQVc7R0FDWjtFQUNEO0lBQ0Usd0JBQWdCO1lBQWhCLGdCQUFnQjtJQUNoQixXQUFXO0dBQ1o7Q0FDRjs7QUFURDtFQUNFO0lBQ0UsMkNBQW1DO1lBQW5DLG1DQUFtQztJQUNuQyxXQUFXO0dBQ1o7RUFDRDtJQUNFLHdCQUFnQjtZQUFoQixnQkFBZ0I7SUFDaEIsV0FBVztHQUNaO0NBQ0Y7O0FBRUQ7RUFDRSxvQkFBb0I7RUFDcEIseUJBQXlCO0NBQzFCOztBQUVEO0VBQ0UsWUFBWTtDQUNiOztBQUVEO0VBQ0UsbUJBQW1CO0VBQ25CLDBCQUEwQjtDQUMzQjs7QUFFRDtFQUNFLFdBQVc7Q0FDWjs7QUFFRDtFQUNFLG1CQUFtQjtFQUNuQixVQUFVO0VBQ1YsbUJBQW1CO0VBQ25CLGdCQUFnQjtFQUNoQixXQUFXO0VBQ1gsaUJBQWlCO0VBQ2pCLGFBQWE7RUFDYixtQkFBbUI7RUFDbkIsd0JBQXdCO0VBQ3hCLGFBQWE7RUFDYixhQUFhO0VBQ2IsaUJBQWlCO0VBQ2pCLGlCQUFpQjtFQUNqQix5RUFBeUU7Q0FDMUU7O0FBRUQ7RUFDRSxlQUFlO0VBQ2YsbUJBQW1CO0VBQ25CLFlBQVk7RUFDWixZQUFZO0VBQ1osYUFBYTtFQUNiLGlCQUFpQjtFQUNqQixjQUFjO0VBQ2QsVUFBVTtFQUNWLDhCQUE4QjtFQUM5QixnQ0FBZ0M7RUFDaEMsa0NBQTBCO1VBQTFCLDBCQUEwQjtFQUMxQixtQkFBbUI7Q0FDcEI7O0FBRUQ7RUFDRSxtQkFBbUI7RUFDbkIsY0FBYztFQUNkLGdCQUFnQjtDQUNqQjs7QUFFRDtFQUNFLHNCQUFzQjtDQUN2Qjs7QUFFRDtFQUNFLGdCQUFnQjtDQUNqQjs7QUFFRDtFQUNFLDJDQUEyQztDQUM1Qzs7QUFFRDtFQUNFLDJDQUEyQztDQUM1Qzs7QUFFRDtFQUNFLG1CQUFtQjtFQUNuQiwwQkFBMEI7RUFDMUIsaUJBQWlCO0VBQ2pCLG1CQUFtQjtFQUNuQixlQUFlO0VBQ2YsbUJBQW1CO0NBQ3BCOztBQUVEO0VBQ0UsdUJBQXVCO0NBQ3hCOztBQUVEO0VBQ0UsZUFBZTtFQUNmLG1CQUFtQjtFQUNuQixlQUFlO0VBQ2YsaUJBQWlCO0VBQ2pCLGVBQWU7RUFDZixpQkFBaUI7Q0FDbEI7O0FBRUQ7RUFDRSxlQUFlO0VBQ2YscUNBQXFDO0VBQ3JDLHNCQUFzQjtDQUN2Qjs7QUFFRDs7OztFQUlFLGlCQUFpQjtFQUNqQixvQkFBb0I7RUFDcEIscURBQXFEO0VBQ3JELGVBQWU7Q0FDaEI7O0FBRUQ7RUFDRSxpQkFBaUI7RUFDakIsb0JBQW9CO0VBQ3BCLHFEQUFxRDtFQUNyRCxlQUFlO0NBQ2hCOztBQUVEO0VBQ0UsZUFBZTtFQUNmLGFBQWE7RUFDYixXQUFXO0VBQ1gsbUJBQW1CO0VBQ25CLDBDQUEwQztFQUMxQyxnQkFBZ0I7Q0FDakI7O0FBRUQ7RUFDRSxZQUFZO0VBQ1osbUJBQW1CO0VBQ25CLGVBQWU7RUFDZixPQUFPO0VBQ1AsYUFBYTtFQUNiLFdBQVc7RUFDWCxpQkFBaUI7RUFDakIsV0FBVztDQUNaOztBQUVEO0VBQ0UsbUJBQW1CO0VBQ25CLDhCQUE4QjtFQUM5QixjQUFjO0VBQ2QsZ0JBQWdCO0VBQ2hCLGVBQWU7RUFDZixlQUFlO0VBQ2YsZUFBZTtDQUNoQjs7QUFFRDtFQUNFLFlBQVk7RUFDWixZQUFZO0VBQ1osbUJBQW1CO0NBQ3BCOztBQUVEO0VBQ0UsWUFBWTtFQUNaLFdBQVc7RUFDWCxnQkFBZ0I7RUFDaEIsa0JBQWtCO0VBQ2xCLG1CQUFtQjtFQUNuQiw4QkFBOEI7RUFDOUIsZUFBZTtFQUNmLGlCQUFpQjtFQUNqQixzQkFBc0I7Q0FDdkI7O0FBRUQ7RUFDRSxZQUFZO0VBQ1osbUJBQW1CO0VBQ25CLGVBQWU7RUFDZixPQUFPO0VBQ1AsYUFBYTtFQUNiLFdBQVc7RUFDWCxpQkFBaUI7RUFDakIsU0FBUztDQUNWOztBQUVEO0VBQ0UsY0FBYztDQUNmOztBQUVEO0VBQ0UsbUJBQW1CO0VBQ25CLGVBQWU7RUFDZixpQkFBaUI7RUFDakIsa0JBQWtCO0NBQ25COztBQUVEO0VBQ0UsZUFBZTtFQUNmLG1CQUFtQjtFQUNuQixrQkFBa0I7RUFDbEIsZUFBZTtDQUNoQjs7QUFFRDtFQUNFLFlBQVk7RUFDWixlQUFlO0VBQ2YsbUJBQW1CO0VBQ25CLGlCQUFpQjtDQUNsQjs7QUFFRDtFQUNFLGNBQWM7Q0FDZjs7QUFFRDtFQUNFLGlCQUFpQjtFQUNqQixlQUFlO0VBQ2YsYUFBYTtFQUNiLGVBQWU7RUFDZiwwQkFBMEI7RUFDMUIsbUJBQW1CO0VBQ25CLCtEQUErRDtDQUNoRTs7QUFFRDtFQUNFLGlCQUFpQjtDQUNsQjs7QUFFRDtFQUNFLGVBQWU7Q0FDaEI7O0FBRUQ7RUFDRSxlQUFlO0NBQ2hCOztBQUVEO0VBQ0U7SUFDRSxlQUFlO0dBQ2hCO0NBQ0Y7O0FBRUQ7RUFDRTtJQUNFLHNCQUFzQjtJQUN0QixZQUFZO0lBQ1osaUJBQWlCO0lBQ2pCLFlBQVk7SUFDWixXQUFXO0lBQ1gsZUFBZTtJQUNmLGlCQUFpQjtJQUNqQixrQkFBa0I7SUFDbEIsaUJBQWlCO0lBQ2pCLGFBQWE7R0FDZDtFQUNEO0lBQ0UsY0FBYztHQUNmO0VBQ0Q7SUFDRSxhQUFhO0dBQ2Q7RUFDRDtJQUNFLHNCQUFzQjtJQUN0QixZQUFZO0lBQ1osaUJBQWlCO0lBQ2pCLFlBQVk7SUFDWixXQUFXO0dBQ1o7RUFDRDtJQUNFLGNBQWM7R0FDZjtDQUNGOztBQUVEO0VBQ0UsOEJBQThCO0VBQzlCLGFBQWE7RUFDYixVQUFVO0NBQ1g7O0FBRUQ7RUFDRSxZQUFZO0VBQ1osV0FBVztDQUNaOztBQUVEO0VBQ0UsY0FBYztDQUNmOztBQUVEO0VBQ0UsVUFBVTtFQUNWLFdBQVc7RUFDWCxlQUFlO0VBQ2YsWUFBWTtFQUNaLGFBQWE7Q0FDZDs7QUFFRDtFQUNFLGFBQWE7RUFDYixrQkFBa0I7Q0FDbkI7O0FBRUQ7RUFDRSxhQUFhO0VBQ2Isa0JBQWtCO0NBQ25COztBQUVEO0VBQ0UsNFVBQTRVO0VBQzVVLFlBQVk7RUFDWixZQUFZO0VBQ1osYUFBYTtFQUNiLHNCQUFzQjtDQUN2Qjs7QUFFRDtFQUNFLFlBQVk7RUFDWixZQUFZO0VBQ1osVUFBVTtFQUNWLFdBQVc7Q0FDWjs7QUFFRDtFQUNFLHlCQUF5QjtDQUMxQjs7QUFFRDtFQUNFLFVBQVU7RUFDVixlQUFlO0VBQ2YsaUJBQWlCO0VBQ2pCLG9CQUFvQjtDQUNyQjs7QUFFRDtFQUNFLGFBQWE7RUFDYixrQkFBa0I7RUFDbEIsZUFBZTtFQUNmLHNCQUFzQjtDQUN2Qjs7QUFFRDtFQUNFLGdCQUFnQjtFQUNoQixlQUFlO0VBQ2YsbUJBQW1CO0VBQ25CLHVCQUF1QjtFQUN2QixvQkFBb0I7RUFDcEIsa0JBQWtCO0VBQ2xCLGFBQWE7Q0FDZDs7QUFFRDtFQUNFLGVBQWU7RUFDZixrQkFBa0I7RUFDbEIsaUJBQWlCO0NBQ2xCOztBQUVEO0VBQ0UsYUFBYTtFQUNiLGFBQWE7RUFDYixjQUFjO0VBQ2QsdUJBQXVCO0VBQ3ZCLGFBQWE7RUFDYixhQUFhO0VBQ2IsZUFBZTtDQUNoQjs7QUFFRDtFQUNFLHcyUEFBdzJQO0VBQ3gyUCw2QkFBNkI7RUFDN0IsNEJBQTRCO0VBQzVCLHNCQUFzQjtFQUN0QixpQkFBaUI7RUFDakIscUJBQXFCO0VBQ3JCLHNCQUFzQjtFQUN0QixZQUFZO0VBQ1osYUFBYTtFQUNiLGVBQWU7Q0FDaEIiLCJmaWxlIjoiZG9jc2VhcmNoLmNzcyIsInNvdXJjZXNDb250ZW50IjpbIi5zZWFyY2hib3gge1xuICBkaXNwbGF5OiBpbmxpbmUtYmxvY2s7XG4gIHBvc2l0aW9uOiByZWxhdGl2ZTtcbiAgd2lkdGg6IDIwMHB4O1xuICBoZWlnaHQ6IDMycHggIWltcG9ydGFudDtcbiAgd2hpdGUtc3BhY2U6IG5vd3JhcDtcbiAgYm94LXNpemluZzogYm9yZGVyLWJveDtcbiAgdmlzaWJpbGl0eTogdmlzaWJsZSAhaW1wb3J0YW50O1xufVxuXG4uc2VhcmNoYm94IC5hbGdvbGlhLWF1dG9jb21wbGV0ZSB7XG4gIGRpc3BsYXk6IGJsb2NrO1xuICB3aWR0aDogMTAwJTtcbiAgaGVpZ2h0OiAxMDAlO1xufVxuXG4uc2VhcmNoYm94X193cmFwcGVyIHtcbiAgd2lkdGg6IDEwMCU7XG4gIGhlaWdodDogMTAwJTtcbiAgei1pbmRleDogOTk5O1xuICBwb3NpdGlvbjogcmVsYXRpdmU7XG59XG5cbi5zZWFyY2hib3hfX2lucHV0IHtcbiAgZGlzcGxheTogaW5saW5lLWJsb2NrO1xuICBib3gtc2l6aW5nOiBib3JkZXItYm94O1xuICB0cmFuc2l0aW9uOiBib3gtc2hhZG93IDAuNHMgZWFzZSwgYmFja2dyb3VuZCAwLjRzIGVhc2U7XG4gIGJvcmRlcjogMDtcbiAgYm9yZGVyLXJhZGl1czogMTZweDtcbiAgYm94LXNoYWRvdzogaW5zZXQgMCAwIDAgMXB4ICNjY2NjY2M7XG4gIGJhY2tncm91bmQ6ICNmZmZmZmYgIWltcG9ydGFudDtcbiAgcGFkZGluZzogMDtcbiAgcGFkZGluZy1yaWdodDogMjZweDtcbiAgcGFkZGluZy1sZWZ0OiAzMnB4O1xuICB3aWR0aDogMTAwJTtcbiAgaGVpZ2h0OiAxMDAlO1xuICB2ZXJ0aWNhbC1hbGlnbjogbWlkZGxlO1xuICB3aGl0ZS1zcGFjZTogbm9ybWFsO1xuICBmb250LXNpemU6IDEycHg7XG4gIGFwcGVhcmFuY2U6IG5vbmU7XG59XG5cbi5zZWFyY2hib3hfX2lucHV0Ojotd2Via2l0LXNlYXJjaC1kZWNvcmF0aW9uLCAuc2VhcmNoYm94X19pbnB1dDo6LXdlYmtpdC1zZWFyY2gtY2FuY2VsLWJ1dHRvbiwgLnNlYXJjaGJveF9faW5wdXQ6Oi13ZWJraXQtc2VhcmNoLXJlc3VsdHMtYnV0dG9uLCAuc2VhcmNoYm94X19pbnB1dDo6LXdlYmtpdC1zZWFyY2gtcmVzdWx0cy1kZWNvcmF0aW9uIHtcbiAgZGlzcGxheTogbm9uZTtcbn1cblxuLnNlYXJjaGJveF9faW5wdXQ6aG92ZXIge1xuICBib3gtc2hhZG93OiBpbnNldCAwIDAgMCAxcHggI2IzYjNiMztcbn1cblxuLnNlYXJjaGJveF9faW5wdXQ6Zm9jdXMsIC5zZWFyY2hib3hfX2lucHV0OmFjdGl2ZSB7XG4gIG91dGxpbmU6IDA7XG4gIGJveC1zaGFkb3c6IGluc2V0IDAgMCAwIDFweCAjYWFhYWFhO1xuICBiYWNrZ3JvdW5kOiAjZmZmZmZmO1xufVxuXG4uc2VhcmNoYm94X19pbnB1dDo6cGxhY2Vob2xkZXIge1xuICBjb2xvcjogI2FhYWFhYTtcbn1cblxuLnNlYXJjaGJveF9fc3VibWl0IHtcbiAgcG9zaXRpb246IGFic29sdXRlO1xuICB0b3A6IDA7XG4gIG1hcmdpbjogMDtcbiAgYm9yZGVyOiAwO1xuICBib3JkZXItcmFkaXVzOiAxNnB4IDAgMCAxNnB4O1xuICBiYWNrZ3JvdW5kLWNvbG9yOiByZ2JhKDY5LCAxNDIsIDIyNSwgMCk7XG4gIHBhZGRpbmc6IDA7XG4gIHdpZHRoOiAzMnB4O1xuICBoZWlnaHQ6IDEwMCU7XG4gIHZlcnRpY2FsLWFsaWduOiBtaWRkbGU7XG4gIHRleHQtYWxpZ246IGNlbnRlcjtcbiAgZm9udC1zaXplOiBpbmhlcml0O1xuICB1c2VyLXNlbGVjdDogbm9uZTtcbiAgcmlnaHQ6IGluaGVyaXQ7XG4gIGxlZnQ6IDA7XG59XG5cbi5zZWFyY2hib3hfX3N1Ym1pdDo6YmVmb3JlIHtcbiAgZGlzcGxheTogaW5saW5lLWJsb2NrO1xuICBtYXJnaW4tcmlnaHQ6IC00cHg7XG4gIGhlaWdodDogMTAwJTtcbiAgdmVydGljYWwtYWxpZ246IG1pZGRsZTtcbiAgY29udGVudDogJyc7XG59XG5cbi5zZWFyY2hib3hfX3N1Ym1pdDpob3ZlciwgLnNlYXJjaGJveF9fc3VibWl0OmFjdGl2ZSB7XG4gIGN1cnNvcjogcG9pbnRlcjtcbn1cblxuLnNlYXJjaGJveF9fc3VibWl0OmZvY3VzIHtcbiAgb3V0bGluZTogMDtcbn1cblxuLnNlYXJjaGJveF9fc3VibWl0IHN2ZyB7XG4gIHdpZHRoOiAxNHB4O1xuICBoZWlnaHQ6IDE0cHg7XG4gIHZlcnRpY2FsLWFsaWduOiBtaWRkbGU7XG4gIGZpbGw6ICM2ZDdlOTY7XG59XG5cbi5zZWFyY2hib3hfX3Jlc2V0IHtcbiAgZGlzcGxheTogYmxvY2s7XG4gIHBvc2l0aW9uOiBhYnNvbHV0ZTtcbiAgdG9wOiA4cHg7XG4gIHJpZ2h0OiA4cHg7XG4gIG1hcmdpbjogMDtcbiAgYm9yZGVyOiAwO1xuICBiYWNrZ3JvdW5kOiBub25lO1xuICBjdXJzb3I6IHBvaW50ZXI7XG4gIHBhZGRpbmc6IDA7XG4gIGZvbnQtc2l6ZTogaW5oZXJpdDtcbiAgdXNlci1zZWxlY3Q6IG5vbmU7XG4gIGZpbGw6IHJnYmEoMCwgMCwgMCwgMC41KTtcbn1cblxuLnNlYXJjaGJveF9fcmVzZXQuaGlkZSB7XG4gIGRpc3BsYXk6IG5vbmU7XG59XG5cbi5zZWFyY2hib3hfX3Jlc2V0OmZvY3VzIHtcbiAgb3V0bGluZTogMDtcbn1cblxuLnNlYXJjaGJveF9fcmVzZXQgc3ZnIHtcbiAgZGlzcGxheTogYmxvY2s7XG4gIG1hcmdpbjogNHB4O1xuICB3aWR0aDogOHB4O1xuICBoZWlnaHQ6IDhweDtcbn1cblxuLnNlYXJjaGJveF9faW5wdXQ6dmFsaWQgfiAuc2VhcmNoYm94X19yZXNldCB7XG4gIGRpc3BsYXk6IGJsb2NrO1xuICBhbmltYXRpb24tbmFtZTogc2J4LXJlc2V0LWluO1xuICBhbmltYXRpb24tZHVyYXRpb246IDAuMTVzO1xufVxuXG5Aa2V5ZnJhbWVzIHNieC1yZXNldC1pbiB7XG4gIDAlIHtcbiAgICB0cmFuc2Zvcm06IHRyYW5zbGF0ZTNkKC0yMCUsIDAsIDApO1xuICAgIG9wYWNpdHk6IDA7XG4gIH1cbiAgMTAwJSB7XG4gICAgdHJhbnNmb3JtOiBub25lO1xuICAgIG9wYWNpdHk6IDE7XG4gIH1cbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlLmFsZ29saWEtYXV0b2NvbXBsZXRlLXJpZ2h0IC5kcy1kcm9wZG93bi1tZW51IHtcbiAgcmlnaHQ6IDAgIWltcG9ydGFudDtcbiAgbGVmdDogaW5oZXJpdCAhaW1wb3J0YW50O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUuYWxnb2xpYS1hdXRvY29tcGxldGUtcmlnaHQgLmRzLWRyb3Bkb3duLW1lbnU6YmVmb3JlIHtcbiAgcmlnaHQ6IDQ4cHg7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZS5hbGdvbGlhLWF1dG9jb21wbGV0ZS1sZWZ0IC5kcy1kcm9wZG93bi1tZW51IHtcbiAgbGVmdDogMCAhaW1wb3J0YW50O1xuICByaWdodDogaW5oZXJpdCAhaW1wb3J0YW50O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUuYWxnb2xpYS1hdXRvY29tcGxldGUtbGVmdCAuZHMtZHJvcGRvd24tbWVudTpiZWZvcmUge1xuICBsZWZ0OiA0OHB4O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmRzLWRyb3Bkb3duLW1lbnUge1xuICBwb3NpdGlvbjogcmVsYXRpdmU7XG4gIHRvcDogLTZweDtcbiAgYm9yZGVyLXJhZGl1czogNHB4O1xuICBtYXJnaW46IDZweCAwIDA7XG4gIHBhZGRpbmc6IDA7XG4gIHRleHQtYWxpZ246IGxlZnQ7XG4gIGhlaWdodDogYXV0bztcbiAgcG9zaXRpb246IHJlbGF0aXZlO1xuICBiYWNrZ3JvdW5kOiB0cmFuc3BhcmVudDtcbiAgYm9yZGVyOiBub25lO1xuICB6LWluZGV4OiA5OTk7XG4gIG1heC13aWR0aDogNjAwcHg7XG4gIG1pbi13aWR0aDogNTAwcHg7XG4gIGJveC1zaGFkb3c6IDAgMXB4IDAgMCByZ2JhKDAsIDAsIDAsIDAuMiksIDAgMnB4IDNweCAwIHJnYmEoMCwgMCwgMCwgMC4xKTtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5kcy1kcm9wZG93bi1tZW51OmJlZm9yZSB7XG4gIGRpc3BsYXk6IGJsb2NrO1xuICBwb3NpdGlvbjogYWJzb2x1dGU7XG4gIGNvbnRlbnQ6ICcnO1xuICB3aWR0aDogMTRweDtcbiAgaGVpZ2h0OiAxNHB4O1xuICBiYWNrZ3JvdW5kOiAjZmZmO1xuICB6LWluZGV4OiAxMDAwO1xuICB0b3A6IC03cHg7XG4gIGJvcmRlci10b3A6IDFweCBzb2xpZCAjZDlkOWQ5O1xuICBib3JkZXItcmlnaHQ6IDFweCBzb2xpZCAjZDlkOWQ5O1xuICB0cmFuc2Zvcm06IHJvdGF0ZSgtNDVkZWcpO1xuICBib3JkZXItcmFkaXVzOiAycHg7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuZHMtZHJvcGRvd24tbWVudSAuZHMtc3VnZ2VzdGlvbnMge1xuICBwb3NpdGlvbjogcmVsYXRpdmU7XG4gIHotaW5kZXg6IDEwMDA7XG4gIG1hcmdpbi10b3A6IDhweDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5kcy1kcm9wZG93bi1tZW51IC5kcy1zdWdnZXN0aW9ucyBhOmhvdmVyIHtcbiAgdGV4dC1kZWNvcmF0aW9uOiBub25lO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmRzLWRyb3Bkb3duLW1lbnUgLmRzLXN1Z2dlc3Rpb24ge1xuICBjdXJzb3I6IHBvaW50ZXI7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuZHMtZHJvcGRvd24tbWVudSAuZHMtc3VnZ2VzdGlvbi5kcy1jdXJzb3IgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24uc3VnZ2VzdGlvbi1sYXlvdXQtc2ltcGxlIHtcbiAgYmFja2dyb3VuZC1jb2xvcjogcmdiYSg2OSwgMTQyLCAyMjUsIDAuMDUpO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmRzLWRyb3Bkb3duLW1lbnUgLmRzLXN1Z2dlc3Rpb24uZHMtY3Vyc29yIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uOm5vdCguc3VnZ2VzdGlvbi1sYXlvdXQtc2ltcGxlKSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY29udGVudCB7XG4gIGJhY2tncm91bmQtY29sb3I6IHJnYmEoNjksIDE0MiwgMjI1LCAwLjA1KTtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5kcy1kcm9wZG93bi1tZW51IFtjbGFzc149J2RzLWRhdGFzZXQtJ10ge1xuICBwb3NpdGlvbjogcmVsYXRpdmU7XG4gIGJvcmRlcjogc29saWQgMXB4ICNkOWQ5ZDk7XG4gIGJhY2tncm91bmQ6ICNmZmY7XG4gIGJvcmRlci1yYWRpdXM6IDRweDtcbiAgb3ZlcmZsb3c6IGF1dG87XG4gIHBhZGRpbmc6IDAgOHB4IDhweDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5kcy1kcm9wZG93bi1tZW51ICoge1xuICBib3gtc2l6aW5nOiBib3JkZXItYm94O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24ge1xuICBkaXNwbGF5OiBibG9jaztcbiAgcG9zaXRpb246IHJlbGF0aXZlO1xuICBwYWRkaW5nOiAwIDhweDtcbiAgYmFja2dyb3VuZDogI2ZmZjtcbiAgY29sb3I6ICMwMjA2MGM7XG4gIG92ZXJmbG93OiBoaWRkZW47XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0taGlnaGxpZ2h0IHtcbiAgY29sb3I6ICMxNzRkOGM7XG4gIGJhY2tncm91bmQ6IHJnYmEoMTQzLCAxODcsIDIzNywgMC4xKTtcbiAgcGFkZGluZzogMC4xZW0gMC4wNWVtO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNhdGVnb3J5LWhlYWRlciAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyLWx2bDBcbi5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1oaWdobGlnaHQsXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNhdGVnb3J5LWhlYWRlciAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyLWx2bDFcbi5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1oaWdobGlnaHQge1xuICBwYWRkaW5nOiAwIDAgMXB4O1xuICBiYWNrZ3JvdW5kOiBpbmhlcml0O1xuICBib3gtc2hhZG93OiBpbnNldCAwIC0ycHggMCAwIHJnYmEoNjksIDE0MiwgMjI1LCAwLjgpO1xuICBjb2xvcjogaW5oZXJpdDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS10ZXh0IC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1oaWdobGlnaHQge1xuICBwYWRkaW5nOiAwIDAgMXB4O1xuICBiYWNrZ3JvdW5kOiBpbmhlcml0O1xuICBib3gtc2hhZG93OiBpbnNldCAwIC0ycHggMCAwIHJnYmEoNjksIDE0MiwgMjI1LCAwLjgpO1xuICBjb2xvcjogaW5oZXJpdDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1jb250ZW50IHtcbiAgZGlzcGxheTogYmxvY2s7XG4gIGZsb2F0OiByaWdodDtcbiAgd2lkdGg6IDcwJTtcbiAgcG9zaXRpb246IHJlbGF0aXZlO1xuICBwYWRkaW5nOiA1LjMzMzMzcHggMCA1LjMzMzMzcHggMTAuNjY2NjdweDtcbiAgY3Vyc29yOiBwb2ludGVyO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNvbnRlbnQ6YmVmb3JlIHtcbiAgY29udGVudDogJyc7XG4gIHBvc2l0aW9uOiBhYnNvbHV0ZTtcbiAgZGlzcGxheTogYmxvY2s7XG4gIHRvcDogMDtcbiAgaGVpZ2h0OiAxMDAlO1xuICB3aWR0aDogMXB4O1xuICBiYWNrZ3JvdW5kOiAjZGRkO1xuICBsZWZ0OiAtMXB4O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNhdGVnb3J5LWhlYWRlciB7XG4gIHBvc2l0aW9uOiByZWxhdGl2ZTtcbiAgYm9yZGVyLWJvdHRvbTogMXB4IHNvbGlkICNkZGQ7XG4gIGRpc3BsYXk6IG5vbmU7XG4gIG1hcmdpbi10b3A6IDhweDtcbiAgcGFkZGluZzogNHB4IDA7XG4gIGZvbnQtc2l6ZTogMWVtO1xuICBjb2xvcjogIzMzMzYzZDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS13cmFwcGVyIHtcbiAgd2lkdGg6IDEwMCU7XG4gIGZsb2F0OiBsZWZ0O1xuICBwYWRkaW5nOiA4cHggMCAwIDA7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktY29sdW1uIHtcbiAgZmxvYXQ6IGxlZnQ7XG4gIHdpZHRoOiAzMCU7XG4gIHBhZGRpbmctbGVmdDogMDtcbiAgdGV4dC1hbGlnbjogcmlnaHQ7XG4gIHBvc2l0aW9uOiByZWxhdGl2ZTtcbiAgcGFkZGluZzogNS4zMzMzM3B4IDEwLjY2NjY3cHg7XG4gIGNvbG9yOiAjYTRhN2FlO1xuICBmb250LXNpemU6IDAuOWVtO1xuICB3b3JkLXdyYXA6IGJyZWFrLXdvcmQ7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktY29sdW1uOmJlZm9yZSB7XG4gIGNvbnRlbnQ6ICcnO1xuICBwb3NpdGlvbjogYWJzb2x1dGU7XG4gIGRpc3BsYXk6IGJsb2NrO1xuICB0b3A6IDA7XG4gIGhlaWdodDogMTAwJTtcbiAgd2lkdGg6IDFweDtcbiAgYmFja2dyb3VuZDogI2RkZDtcbiAgcmlnaHQ6IDA7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktaW5saW5lIHtcbiAgZGlzcGxheTogbm9uZTtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS10aXRsZSB7XG4gIG1hcmdpbi1ib3R0b206IDRweDtcbiAgY29sb3I6ICMwMjA2MGM7XG4gIGZvbnQtc2l6ZTogMC45ZW07XG4gIGZvbnQtd2VpZ2h0OiBib2xkO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLXRleHQge1xuICBkaXNwbGF5OiBibG9jaztcbiAgbGluZS1oZWlnaHQ6IDEuMmVtO1xuICBmb250LXNpemU6IDAuODVlbTtcbiAgY29sb3I6ICM2MzY3NmQ7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tbm8tcmVzdWx0cyB7XG4gIHdpZHRoOiAxMDAlO1xuICBwYWRkaW5nOiA4cHggMDtcbiAgdGV4dC1hbGlnbjogY2VudGVyO1xuICBmb250LXNpemU6IDEuMmVtO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLW5vLXJlc3VsdHM6OmJlZm9yZSB7XG4gIGRpc3BsYXk6IG5vbmU7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiBjb2RlIHtcbiAgcGFkZGluZzogMXB4IDVweDtcbiAgZm9udC1zaXplOiA5MCU7XG4gIGJvcmRlcjogbm9uZTtcbiAgY29sb3I6ICMyMjIyMjI7XG4gIGJhY2tncm91bmQtY29sb3I6ICNlYmViZWI7XG4gIGJvcmRlci1yYWRpdXM6IDNweDtcbiAgZm9udC1mYW1pbHk6IE1lbmxvLCBNb25hY28sIENvbnNvbGFzLCAnQ291cmllciBOZXcnLCBtb25vc3BhY2U7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiBjb2RlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1oaWdobGlnaHQge1xuICBiYWNrZ3JvdW5kOiBub25lO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24uYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbl9fbWFpbiAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyIHtcbiAgZGlzcGxheTogYmxvY2s7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uX19zZWNvbmRhcnkge1xuICBkaXNwbGF5OiBibG9jaztcbn1cblxuQG1lZGlhIGFsbCBhbmQgKG1pbi13aWR0aDogNzY4cHgpIHtcbiAgLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1zdWJjYXRlZ29yeS1jb2x1bW4ge1xuICAgIGRpc3BsYXk6IGJsb2NrO1xuICB9XG59XG5cbkBtZWRpYSBhbGwgYW5kIChtYXgtd2lkdGg6IDc2OHB4KSB7XG4gIC5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktY29sdW1uIHtcbiAgICBkaXNwbGF5OiBpbmxpbmUtYmxvY2s7XG4gICAgd2lkdGg6IGF1dG87XG4gICAgdGV4dC1hbGlnbjogbGVmdDtcbiAgICBmbG9hdDogbGVmdDtcbiAgICBwYWRkaW5nOiAwO1xuICAgIGNvbG9yOiAjMDIwNjBjO1xuICAgIGZvbnQtc2l6ZTogMC45ZW07XG4gICAgZm9udC13ZWlnaHQ6IGJvbGQ7XG4gICAgdGV4dC1hbGlnbjogbGVmdDtcbiAgICBvcGFjaXR5OiAwLjU7XG4gIH1cbiAgLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1zdWJjYXRlZ29yeS1jb2x1bW46YmVmb3JlIHtcbiAgICBkaXNwbGF5OiBub25lO1xuICB9XG4gIC5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktY29sdW1uOmFmdGVyIHtcbiAgICBjb250ZW50OiAnfCc7XG4gIH1cbiAgLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1jb250ZW50IHtcbiAgICBkaXNwbGF5OiBpbmxpbmUtYmxvY2s7XG4gICAgd2lkdGg6IGF1dG87XG4gICAgdGV4dC1hbGlnbjogbGVmdDtcbiAgICBmbG9hdDogbGVmdDtcbiAgICBwYWRkaW5nOiAwO1xuICB9XG4gIC5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY29udGVudDpiZWZvcmUge1xuICAgIGRpc3BsYXk6IG5vbmU7XG4gIH1cbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5zdWdnZXN0aW9uLWxheW91dC1zaW1wbGUuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbiB7XG4gIGJvcmRlci1ib3R0b206IHNvbGlkIDFweCAjZWVlO1xuICBwYWRkaW5nOiA4cHg7XG4gIG1hcmdpbjogMDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5zdWdnZXN0aW9uLWxheW91dC1zaW1wbGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNvbnRlbnQge1xuICB3aWR0aDogMTAwJTtcbiAgcGFkZGluZzogMDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5zdWdnZXN0aW9uLWxheW91dC1zaW1wbGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWNvbnRlbnQ6OmJlZm9yZSB7XG4gIGRpc3BsYXk6IG5vbmU7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuc3VnZ2VzdGlvbi1sYXlvdXQtc2ltcGxlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS1jYXRlZ29yeS1oZWFkZXIge1xuICBtYXJnaW46IDA7XG4gIHBhZGRpbmc6IDA7XG4gIGRpc3BsYXk6IGJsb2NrO1xuICB3aWR0aDogMTAwJTtcbiAgYm9yZGVyOiBub25lO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyLWx2bDAge1xuICBvcGFjaXR5OiAwLjY7XG4gIGZvbnQtc2l6ZTogMC44NWVtO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyLWx2bDEge1xuICBvcGFjaXR5OiAwLjY7XG4gIGZvbnQtc2l6ZTogMC44NWVtO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tY2F0ZWdvcnktaGVhZGVyLWx2bDE6OmJlZm9yZSB7XG4gIGJhY2tncm91bmQtaW1hZ2U6IHVybCgnZGF0YTppbWFnZS9zdmcreG1sO3V0ZjgsPHN2ZyB3aWR0aD1cIjEwXCIgaGVpZ2h0PVwiMTBcIiB2aWV3Qm94PVwiMCAwIDIwIDM4XCIgeG1sbnM9XCJodHRwOi8vd3d3LnczLm9yZy8yMDAwL3N2Z1wiPjxwYXRoIGQ9XCJNMS40OSA0LjMxbDE0IDE2LjEyNi4wMDItMi42MjQtMTQgMTYuMDc0LTEuMzE0IDEuNTEgMy4wMTcgMi42MjYgMS4zMTMtMS41MDggMTQtMTYuMDc1IDEuMTQyLTEuMzEzLTEuMTQtMS4zMTMtMTQtMTYuMTI1TDMuMi4xOC4xOCAyLjhsMS4zMSAxLjUxelwiIGZpbGwtcnVsZT1cImV2ZW5vZGRcIiBmaWxsPVwiJTIzMUQzNjU3XCIgLz48L3N2Zz4nKTtcbiAgY29udGVudDogJyc7XG4gIHdpZHRoOiAxMHB4O1xuICBoZWlnaHQ6IDEwcHg7XG4gIGRpc3BsYXk6IGlubGluZS1ibG9jaztcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5zdWdnZXN0aW9uLWxheW91dC1zaW1wbGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLXdyYXBwZXIge1xuICB3aWR0aDogMTAwJTtcbiAgZmxvYXQ6IGxlZnQ7XG4gIG1hcmdpbjogMDtcbiAgcGFkZGluZzogMDtcbn1cblxuLmFsZ29saWEtYXV0b2NvbXBsZXRlIC5zdWdnZXN0aW9uLWxheW91dC1zaW1wbGUgLmFsZ29saWEtZG9jc2VhcmNoLXN1Z2dlc3Rpb24tLWR1cGxpY2F0ZS1jb250ZW50LCAuYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tc3ViY2F0ZWdvcnktaW5saW5lIHtcbiAgZGlzcGxheTogbm9uZSAhaW1wb3J0YW50O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tdGl0bGUge1xuICBtYXJnaW46IDA7XG4gIGNvbG9yOiAjNDU4ZWUxO1xuICBmb250LXNpemU6IDAuOWVtO1xuICBmb250LXdlaWdodDogbm9ybWFsO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tdGl0bGU6OmJlZm9yZSB7XG4gIGNvbnRlbnQ6ICcjJztcbiAgZm9udC13ZWlnaHQ6IGJvbGQ7XG4gIGNvbG9yOiAjNDU4ZWUxO1xuICBkaXNwbGF5OiBpbmxpbmUtYmxvY2s7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuc3VnZ2VzdGlvbi1sYXlvdXQtc2ltcGxlIC5hbGdvbGlhLWRvY3NlYXJjaC1zdWdnZXN0aW9uLS10ZXh0IHtcbiAgbWFyZ2luOiA0cHggMCAwO1xuICBkaXNwbGF5OiBibG9jaztcbiAgbGluZS1oZWlnaHQ6IDEuNGVtO1xuICBwYWRkaW5nOiA1LjMzMzMzcHggOHB4O1xuICBiYWNrZ3JvdW5kOiAjZjhmOGY4O1xuICBmb250LXNpemU6IDAuODVlbTtcbiAgb3BhY2l0eTogMC44O1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLnN1Z2dlc3Rpb24tbGF5b3V0LXNpbXBsZSAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0tdGV4dCAuYWxnb2xpYS1kb2NzZWFyY2gtc3VnZ2VzdGlvbi0taGlnaGxpZ2h0IHtcbiAgY29sb3I6ICMzZjQxNDU7XG4gIGZvbnQtd2VpZ2h0OiBib2xkO1xuICBib3gtc2hhZG93OiBub25lO1xufVxuXG4uYWxnb2xpYS1hdXRvY29tcGxldGUgLmFsZ29saWEtZG9jc2VhcmNoLWZvb3RlciB7XG4gIHdpZHRoOiAxMzRweDtcbiAgaGVpZ2h0OiAyMHB4O1xuICB6LWluZGV4OiAyMDAwO1xuICBtYXJnaW4tdG9wOiAxMC42NjY2N3B4O1xuICBmbG9hdDogcmlnaHQ7XG4gIGZvbnQtc2l6ZTogMDtcbiAgbGluZS1oZWlnaHQ6IDA7XG59XG5cbi5hbGdvbGlhLWF1dG9jb21wbGV0ZSAuYWxnb2xpYS1kb2NzZWFyY2gtZm9vdGVyLS1sb2dvIHtcbiAgYmFja2dyb3VuZC1pbWFnZTogdXJsKFwiZGF0YTppbWFnZS9zdmcreG1sLCUzQ3N2ZyB3aWR0aD0nMTY4JyBoZWlnaHQ9JzI0JyB4bWxucz0naHR0cDovL3d3dy53My5vcmcvMjAwMC9zdmcnJTNFJTNDZyBmaWxsPSdub25lJyBmaWxsLXJ1bGU9J2V2ZW5vZGQnJTNFJTNDcGF0aCBkPSdNNzguOTg4LjkzOGgxNi41OTRhMi45NjggMi45NjggMCAwIDEgMi45NjYgMi45NjZWMjAuNWEyLjk2NyAyLjk2NyAwIDAgMS0yLjk2NiAyLjk2NEg3OC45ODhhMi45NjcgMi45NjcgMCAwIDEtMi45NjYtMi45NjRWMy44OTdBMi45NjEgMi45NjEgMCAwIDEgNzguOTg4LjkzOHptNDEuOTM3IDE3Ljg2NmMtNC4zODYuMDItNC4zODYtMy41NC00LjM4Ni00LjEwNmwtLjAwNy0xMy4zMzYgMi42NzUtLjQyNHYxMy4yNTRjMCAuMzIyIDAgMi4zNTggMS43MTggMi4zNjR2Mi4yNDh6bS0xMC44NDYtMi4xOGMuODIxIDAgMS40My0uMDQ3IDEuODU1LS4xMjl2LTIuNzE5YTYuMzM0IDYuMzM0IDAgMCAwLTEuNTc0LS4xOTljLS4yOTUgMC0uNTk2LjAyMS0uODk3LjA2OWEyLjY5OSAyLjY5OSAwIDAgMC0uODE0LjI0Yy0uMjQuMTE2LS40MzkuMjgtLjU4Mi40OTEtLjE1LjIxMi0uMjE5LjMzNS0uMjE5LjY1NiAwIC42MjguMjE5Ljk5MS42MTYgMS4yM3MuOTM4LjM2MiAxLjYxNS4zNjJ6bS0uMjMzLTkuN2MuODgzIDAgMS42MjkuMTA5IDIuMjMxLjMyOC42MDIuMjE4IDEuMDg4LjUyNSAxLjQ0NC45MTUuMzYzLjM5Ni42MDkuOTIyLjc2IDEuNDgzLjE1Ny41Ni4yMzIgMS4xNzUuMjMyIDEuODV2Ni44NzRjLS40MS4wODktMS4wMzQuMTktMS44NjguMzE0LS44MzQuMTIzLTEuNzcyLjE4NS0yLjgxMy4xODUtLjY5IDAtMS4zMjctLjA2OS0xLjg5NS0uMTk4YTQuMDAxIDQuMDAxIDAgMCAxLTEuNDcxLS42MzYgMy4wODUgMy4wODUgMCAwIDEtLjk1MS0xLjEzNGMtLjIyNi0uNDY1LS4zNDMtMS4xMi0uMzQzLTEuODAzIDAtLjY1Ni4xMy0xLjA3My4zODQtMS41MjUuMjYtLjQ1LjYwOC0uODE5IDEuMDQ3LTEuMTA2LjQ0NS0uMjg3Ljk1LS40OTIgMS41MzItLjYxNWE4LjggOC44IDAgMCAxIDEuODItLjE4NSA4LjQwNCA4LjQwNCAwIDAgMSAxLjk3Mi4yNHYtLjQzOGMwLS4zMDctLjAzNS0uNi0uMTEtLjg3NGExLjg4IDEuODggMCAwIDAtLjM4NC0uNzMgMS43ODQgMS43ODQgMCAwIDAtLjcyNC0uNDkzIDMuMTY0IDMuMTY0IDAgMCAwLTEuMTQzLS4yMDVjLS42MTYgMC0xLjE3Ny4wNzUtMS42OS4xNjRhNy43MzUgNy43MzUgMCAwIDAtMS4yNi4zMDdsLS4zMjEtMi4xOTJjLjMzNS0uMTE3LjgzNC0uMjMzIDEuNDc4LS4zNDlhMTAuOTggMTAuOTggMCAwIDEgMi4wNzMtLjE3OHptNTIuODQyIDkuNjI2Yy44MjIgMCAxLjQzLS4wNDggMS44NTQtLjEzVjEzLjdhNi4zNDcgNi4zNDcgMCAwIDAtMS41NzQtLjE5OWMtLjI5NCAwLS41OTUuMDIxLS44OTYuMDY5YTIuNyAyLjcgMCAwIDAtLjgxNC4yNCAxLjQ2IDEuNDYgMCAwIDAtLjU4Mi40OTFjLS4xNS4yMTItLjIxOC4zMzUtLjIxOC42NTYgMCAuNjI4LjIxOC45OTEuNjE1IDEuMjMuNDA0LjI0NS45MzguMzYyIDEuNjE1LjM2MnptLS4yMjYtOS42OTRjLjg4MyAwIDEuNjI5LjEwOCAyLjIzMS4zMjcuNjAyLjIxOSAxLjA4OC41MjYgMS40NDQuOTE1LjM1NS4zOS42MDkuOTIzLjc1OSAxLjQ4My4xNTguNTYuMjMzIDEuMTc1LjIzMyAxLjg1MnY2Ljg3M2MtLjQxLjA4OC0xLjAzNC4xOS0xLjg2OC4zMTQtLjgzNC4xMjMtMS43NzIuMTg0LTIuODEzLjE4NC0uNjkgMC0xLjMyNy0uMDY4LTEuODk1LS4xOThhNC4wMDEgNC4wMDEgMCAwIDEtMS40NzEtLjYzNSAzLjA4NSAzLjA4NSAwIDAgMS0uOTUxLTEuMTM0Yy0uMjI2LS40NjUtLjM0My0xLjEyLS4zNDMtMS44MDQgMC0uNjU2LjEzLTEuMDczLjM4NC0xLjUyNC4yNi0uNDUuNjA4LS44MiAxLjA0Ny0xLjEwNy40NDUtLjI4Ni45NS0uNDkxIDEuNTMyLS42MTRhOC44MDMgOC44MDMgMCAwIDEgMi43NTEtLjEzYy4zMjkuMDM0LjY3MS4wOTYgMS4wNC4xODV2LS40MzdhMy4zIDMuMyAwIDAgMC0uMTA5LS44NzUgMS44NzMgMS44NzMgMCAwIDAtLjM4NC0uNzMxIDEuNzg0IDEuNzg0IDAgMCAwLS43MjQtLjQ5MiAzLjE2NSAzLjE2NSAwIDAgMC0xLjE0My0uMjA1Yy0uNjE2IDAtMS4xNzcuMDc1LTEuNjkuMTY0LS41MTQuMDg5LS45MzguMTkxLTEuMjYuMzA3bC0uMzIxLTIuMTkzYy4zMzUtLjExNi44MzQtLjIzMiAxLjQ3OC0uMzQ4YTExLjYzMyAxMS42MzMgMCAwIDEgMi4wNzMtLjE3N3ptLTguMDM0LTEuMjcxYTEuNjI2IDEuNjI2IDAgMCAxLTEuNjI4LTEuNjJjMC0uODk1LjcyNS0xLjYyIDEuNjI4LTEuNjIuOTA0IDAgMS42My43MjUgMS42MyAxLjYyIDAgLjg5NS0uNzMzIDEuNjItMS42MyAxLjYyem0xLjM0OCAxMy4yMmgtMi42ODlWNy4yN2wyLjY5LS40MjN2MTEuOTU2em0tNC43MTQgMGMtNC4zODYuMDItNC4zODYtMy41NC00LjM4Ni00LjEwN2wtLjAwOC0xMy4zMzYgMi42NzYtLjQyNHYxMy4yNTRjMCAuMzIyIDAgMi4zNTggMS43MTggMi4zNjR2Mi4yNDh6bS04LjY5OC01LjkwM2MwLTEuMTU2LS4yNTMtMi4xMTktLjc0Ni0yLjc4OC0uNDkzLS42NzctMS4xODMtMS4wMS0yLjA2Ny0xLjAxLS44ODIgMC0xLjU3NC4zMzMtMi4wNjUgMS4wMS0uNDkzLjY3Ni0uNzMzIDEuNjMyLS43MzMgMi43ODggMCAxLjE2OC4yNDYgMS45NTMuNzQgMi42My40OTIuNjgzIDEuMTgzIDEuMDE4IDIuMDY2IDEuMDE4Ljg4MiAwIDEuNTc0LS4zNDIgMi4wNjctMS4wMTkuNDkyLS42ODMuNzM4LTEuNDYuNzM4LTIuNjN6bTIuNzM3LS4wMDdjMCAuOTAyLS4xMyAxLjU4NC0uMzk3IDIuMzNhNS41MiA1LjUyIDAgMCAxLTEuMTI4IDEuOTA2IDQuOTg2IDQuOTg2IDAgMCAxLTEuNzUyIDEuMjIzYy0uNjg1LjI4Ni0xLjczOS40NS0yLjI2NS40NS0uNTI4LS4wMDYtMS41NzQtLjE1Ny0yLjI1Mi0uNDVhNS4wOTYgNS4wOTYgMCAwIDEtMS43NDQtMS4yMjNjLS40ODctLjUyNy0uODYzLTEuMTYyLTEuMTM3LTEuOTA2YTYuMzQ1IDYuMzQ1IDAgMCAxLS40MS0yLjMzYzAtLjkwMi4xMjMtMS43Ny4zOTctMi41MDhhNS41NTQgNS41NTQgMCAwIDEgMS4xNS0xLjg5MiA1LjEzMyA1LjEzMyAwIDAgMSAxLjc1LTEuMjE2Yy42NzktLjI4NyAxLjQyNS0uNDIzIDIuMjMyLS40MjMuODA4IDAgMS41NTMuMTQyIDIuMjM3LjQyMy42ODUuMjg2IDEuMjc0LjY5IDEuNzUzIDEuMjE2YTUuNjQ0IDUuNjQ0IDAgMCAxIDEuMTM1IDEuODkyYy4yODcuNzM4LjQzMSAxLjYwNi40MzEgMi41MDh6bS0yMC4xMzggMGMwIDEuMTIuMjQ2IDIuMzYzLjczOCAyLjg4Mi40OTMuNTIgMS4xMy43OCAxLjkxLjc4LjQyNCAwIC44MjgtLjA2MiAxLjIwNC0uMTc4LjM3Ny0uMTE2LjY3Ny0uMjUzLjkxNy0uNDE3VjkuMzNhMTAuNDc2IDEwLjQ3NiAwIDAgMC0xLjc2Ni0uMjI2Yy0uOTcxLS4wMjgtMS43MS4zNy0yLjIzIDEuMDA0LS41MTMuNjM2LS43NzMgMS43NS0uNzczIDIuNzg4em03LjQzOCA1LjI3NGMwIDEuODI0LS40NjYgMy4xNTYtMS40MDQgNC4wMDQtLjkzNi44NDYtMi4zNjcgMS4yNy00LjI5NiAxLjI3LS43MDUgMC0yLjE3LS4xMzctMy4zNC0uMzk2bC40MzEtMi4xMThjLjk4LjIwNSAyLjI3Mi4yNiAyLjk1LjI2IDEuMDc0IDAgMS44NC0uMjE5IDIuMjk5LS42NTYuNDU5LS40MzcuNjg0LTEuMDg2LjY4NC0xLjk0OHYtLjQzN2E4LjA3IDguMDcgMCAwIDEtMS4wNDcuMzk3Yy0uNDMuMTMtLjkzLjE5OC0xLjQ5Mi4xOTgtLjczOSAwLTEuNDEtLjExNi0yLjAxOC0uMzQ5YTQuMjA2IDQuMjA2IDAgMCAxLTEuNTY3LTEuMDI1Yy0uNDMxLS40NS0uNzc0LTEuMDE3LTEuMDEzLTEuNjk0LS4yNC0uNjc3LS4zNjMtMS44ODUtLjM2My0yLjc3MyAwLS44MzQuMTMtMS44OC4zODQtMi41NzcuMjYtLjY5Ni42MjktMS4yOTggMS4xMjktMS43OTYuNDkzLS40OTggMS4wOTUtLjg4MSAxLjgtMS4xNjJhNi42MDUgNi42MDUgMCAwIDEgMi40MjgtLjQ1N2MuODcgMCAxLjY3LjEwOSAyLjQ1LjI0Ljc4LjEyOSAxLjQ0NC4yNjUgMS45ODUuNDE1VjE4LjE3eicgZmlsbD0nJTIzNTQ2OEZGJy8lM0UlM0NwYXRoIGQ9J002Ljk3MiA2LjY3N3YxLjYyN2MtLjcxMi0uNDQ2LTEuNTItLjY3LTIuNDI1LS42Ny0uNTg1IDAtMS4wNDUuMTMtMS4zOC4zOTFhMS4yNCAxLjI0IDAgMCAwLS41MDIgMS4wM2MwIC40MjUuMTY0Ljc2NS40OTQgMS4wMi4zMy4yNTYuODM1LjUzMiAxLjUxNi44My40NDcuMTkyLjc5NS4zNTYgMS4wNDUuNDk1LjI1LjEzOC41MzcuMzMyLjg2Mi41ODIuMzI0LjI1LjU2My41NDguNzE4Ljg5NC4xNTQuMzQ1LjIzLjc0MS4yMyAxLjE4OCAwIC45NDctLjMzNCAxLjY5MS0xLjAwNCAyLjIzNC0uNjcuNTQyLTEuNTM3LjgxNC0yLjYwMS44MTQtMS4xOCAwLTIuMTYtLjIyOS0yLjkzNi0uNjg2di0xLjcwOGMuODQuNjI4IDEuODE0Ljk0MiAyLjkyLjk0Mi41ODUgMCAxLjA0OC0uMTM2IDEuMzg4LS40MDcuMzQtLjI3MS41MS0uNjQ2LjUxLTEuMTI1IDAtLjI4Ny0uMS0uNTUtLjMwMi0uNzktLjIwMy0uMjQtLjQyLS40Mi0uNjU1LS41NDItLjIzNC0uMTIzLS41ODUtLjI5LTEuMDUzLS41MDMtLjI3Ni0uMTI3LS40Ny0uMjE4LS41ODItLjI3MWExMy42NyAxMy42NyAwIDAgMS0uNTUtLjI4NyA0LjI3NSA0LjI3NSAwIDAgMS0uNTY3LS4zNTEgNi45MiA2LjkyIDAgMCAxLS40NTUtLjRjLS4xOC0uMTctLjMxLS4zNC0uMzktLjUxLS4wOC0uMTctLjE1NS0uMzctLjIyNC0uNTk4YTIuNTUzIDIuNTUzIDAgMCAxLS4xMDQtLjc0MmMwLS45MTUuMzMzLTEuNjM4Ljk5OC0yLjE3LjY2NC0uNTMyIDEuNTIzLS43OTggMi41NzYtLjc5OC45NjggMCAxLjc5My4xNyAyLjQ3My41MXptNy40NjggNS42OTZ2LS4yODdjLS4wMjItLjYwNy0uMTg3LTEuMDg4LS40OTUtMS40NDQtLjMwOS0uMzU3LS43NS0uNTM1LTEuMzI0LS41MzUtLjUzMiAwLS45OS4xOTQtMS4zNzMuNTgzLS4zODIuMzg4LS42MjIuOTQ5LS43MTcgMS42ODNoMy45MDl6bTEuMDA1IDIuNzkydjEuNDA0Yy0uNTk2LjM0LTEuMzgzLjUxLTIuMzYyLjUxLTEuMjU1IDAtMi4yNTUtLjM3Ny0zLTEuMTMyLS43NDQtLjc1NS0xLjExNi0xLjc0NC0xLjExNi0yLjk2OCAwLTEuMjk3LjM0LTIuMzE2IDEuMDIxLTMuMDU1LjY4LS43NCAxLjU0OC0xLjExIDIuNi0xLjExIDEuMDMzIDAgMS44NTIuMzIzIDIuNDU4Ljk2Ni42MDYuNjQ0LjkxIDEuNTcyLjkxIDIuNzg0IDAgLjMzLS4wMzMuNjc2LS4wOTYgMS4wMzhoLTUuMzE0Yy4xMDcuNzAyLjQwNSAxLjIzOS44OTQgMS42MTEuNDkuMzcyIDEuMTA2LjU1OCAxLjg1LjU1OC44NjIgMCAxLjU4LS4yMDIgMi4xNTUtLjYwNnptNi42MDUtMS43N2gtMS4yMTJjLS41OTYgMC0xLjA0NS4xMTYtMS4zNDkuMzUtLjMwMy4yMzQtLjQ1NC41MzItLjQ1NC44OTQgMCAuMzcyLjExNy42NjQuMzUuODc3LjIzNS4yMTMuNTc1LjMyIDEuMDIyLjMyLjUxIDAgLjkxMi0uMTQyIDEuMjA0LS40MjQuMjkzLS4yODEuNDQtLjY1MS40NC0xLjEwOHYtLjkxem0tNC4wNjgtMi41NTRWOS4zMjVjLjYyNy0uMzYxIDEuNDU3LS41NDIgMi40ODktLjU0MiAyLjExNiAwIDMuMTc1IDEuMDI2IDMuMTc1IDMuMDhWMTdoLTEuNTQ4di0uOTU3Yy0uNDE1LjY4LTEuMTQzIDEuMDItMi4xODYgMS4wMi0uNzY2IDAtMS4zOC0uMjItMS44NDMtLjY2MS0uNDYyLS40NDItLjY5NC0xLjAwMy0uNjk0LTEuNjg0IDAtLjc3Ni4yOTMtMS4zOC44NzgtMS44MS41ODUtLjQzMSAxLjQwNC0uNjQ3IDIuNDU3LS42NDdoMS4zNFYxMS44YzAtLjU1NC0uMTMzLS45NzEtLjM5OS0xLjI1My0uMjY2LS4yODItLjcwNy0uNDIzLTEuMzI0LS40MjNhNC4wNyA0LjA3IDAgMCAwLTIuMzQ1LjcxOHptOS4zMzMtMS45M3YxLjQyYy4zOTQtMSAxLjEwMS0xLjUgMi4xMjMtMS41LjE0OCAwIC4zMTMuMDE2LjQ5NC4wNDh2MS41MzFhMS44ODUgMS44ODUgMCAwIDAtLjc1LS4xNDNjLS41NDIgMC0uOTg5LjI0LTEuMzQuNzE4LS4zNTEuNDc5LS41MjcgMS4wNDgtLjUyNyAxLjcwN1YxN2gtMS41NjNWOC45MWgxLjU2M3ptNS4wMSA0LjA4NGMuMDIyLjgyLjI3MiAxLjQ5Mi43NSAyLjAxOS40NzkuNTI2IDEuMTUuNzkgMi4wMS43OS42MzkgMCAxLjIzNS0uMTc2IDEuNzg4LS41Mjd2MS40MDRjLS41MjEuMzE5LTEuMTg2LjQ3OS0xLjk5NS40NzktMS4yNjUgMC0yLjI3Ni0uNC0zLjAzMS0xLjE5Ny0uNzU1LS43OTgtMS4xMzMtMS43OTItMS4xMzMtMi45ODQgMC0xLjE2LjM4LTIuMTUxIDEuMTQtMi45NzUuNzYxLS44MjUgMS43OS0xLjIzNyAzLjA4OC0xLjIzNy43MDIgMCAxLjM0Ni4xNDkgMS45My40NDd2MS40MzZhMy4yNDIgMy4yNDIgMCAwIDAtMS43Ny0uNDk1Yy0uODQgMC0xLjUxMy4yNjYtMi4wMTkuNzk4LS41MDUuNTMyLS43NTggMS4yMTMtLjc1OCAyLjA0MnpNNDAuMjQgNS43MnY0LjU3OWMuNDU4LTEgMS4yOTMtMS41IDIuNTA1LTEuNS43ODcgMCAxLjQyLjI0NSAxLjg5OS43MzQuNDc5LjQ5LjcxOCAxLjE3LjcxOCAyLjA0MlYxN2gtMS41NjR2LTUuMTA2YzAtLjU1My0uMTQtLjk4LS40MjItMS4yODQtLjI4Mi0uMzAzLS42NTItLjQ1NS0xLjExLS40NTUtLjUzMSAwLTEuMDAyLjIwMi0xLjQxMS42MDYtLjQxLjQwNS0uNjE1IDEuMDIyLS42MTUgMS44NTFWMTdoLTEuNTYzVjUuNzJoMS41NjN6bTE0Ljk2NiAxMC4wMmMuNTk2IDAgMS4wOTYtLjI1MyAxLjUtLjc1OC40MDQtLjUwNi42MDYtMS4xNTcuNjA2LTEuOTU1IDAtLjkxNS0uMjAyLTEuNjItLjYwNi0yLjExNC0uNDA0LS40OTUtLjkyLS43NDItMS41NDgtLjc0Mi0uNTUzIDAtMS4wNS4yMjQtMS40OTEuNjctLjQ0Mi40NDctLjY2MiAxLjEzMy0uNjYyIDIuMDU4IDAgLjk1OC4yMTIgMS42Ny42MzggMi4xMzguNDI1LjQ2OS45NDYuNzAzIDEuNTYzLjcwM3pNNTMuMDA0IDUuNzJ2NC40MmMuNTc0LS44OTQgMS4zODgtMS4zNDEgMi40NC0xLjM0MSAxLjAyMiAwIDEuODU3LjM4MyAyLjUwNiAxLjE0OS42NDkuNzY2Ljk3MyAxLjc4MS45NzMgMy4wNDcgMCAxLjEzOC0uMzA5IDIuMTA5LS45MjUgMi45MTItLjYxNy44MDMtMS40NjMgMS4yMDUtMi41MzcgMS4yMDUtMS4wNzUgMC0xLjg5NC0uNDQ3LTIuNDU3LTEuMzRWMTdoLTEuNThWNS43MmgxLjU4em05LjkwOCAxMS4xMDRsLTMuMjIzLTcuOTEzaDEuNzM5bDEuMDA1IDIuNjMyIDEuMjYgMy40MTVjLjA5Ni0uMzIuNDgtMS40NTggMS4xNS0zLjQxNWwuOTA5LTIuNjMyaDEuNjZsLTIuOTIgNy44NjZjLS43NzcgMi4wNzQtMS45NjMgMy4xMS0zLjU1OSAzLjExYTIuOTIgMi45MiAwIDAgMS0uNzM0LS4wNzl2LTEuMzRjLjE3LjA0Mi4zNTEuMDY0LjU0My4wNjQgMS4wMzIgMCAxLjc1NS0uNTcgMi4xNy0xLjcwOHonIGZpbGw9JyUyMzVENjQ5NCcvJTNFJTNDcGF0aCBkPSdNODkuNjMyIDUuOTY3di0uNzcyYS45NzguOTc4IDAgMCAwLS45NzgtLjk3N2gtMi4yOGEuOTc4Ljk3OCAwIDAgMC0uOTc4Ljk3N3YuNzkzYzAgLjA4OC4wODIuMTUuMTcxLjEzYTcuMTI3IDcuMTI3IDAgMCAxIDEuOTg0LS4yOGMuNjUgMCAxLjI5NS4wODggMS45MTcuMjU5LjA4Mi4wMi4xNjQtLjA0LjE2NC0uMTNtLTYuMjQ4IDEuMDFsLS4zOS0uMzg5YS45NzcuOTc3IDAgMCAwLTEuMzgyIDBsLS40NjUuNDY1YS45NzMuOTczIDAgMCAwIDAgMS4zOGwuMzgzLjM4M2MuMDYyLjA2MS4xNS4wNDcuMjA1LS4wMTQuMjI2LS4zMDcuNDcyLS42MDEuNzQ2LS44NzQuMjgxLS4yOC41NjgtLjUyNi44ODMtLjc1MS4wNjgtLjA0Mi4wNzUtLjEzNy4wMi0uMm00LjE2IDIuNDUzdjMuMzQxYzAgLjA5Ni4xMDQuMTY1LjE5Mi4xMTdsMi45Ny0xLjUzN2MuMDY4LS4wMzQuMDg5LS4xMTcuMDU1LS4xODRhMy42OTUgMy42OTUgMCAwIDAtMy4wOC0xLjg2NmMtLjA2OCAwLS4xMzYuMDU0LS4xMzYuMTNtMCA4LjA0OGE0LjQ4OSA0LjQ4OSAwIDAgMS00LjQ5LTQuNDgyIDQuNDg4IDQuNDg4IDAgMCAxIDQuNDktNC40ODIgNC40ODggNC40ODggMCAwIDEgNC40ODkgNC40ODIgNC40ODQgNC40ODQgMCAwIDEtNC40OSA0LjQ4Mm0wLTEwLjg1YTYuMzYzIDYuMzYzIDAgMSAwIDAgMTIuNzI5YzMuNTE4IDAgNi4zNzItMi44NSA2LjM3Mi02LjM2OGE2LjM1OCA2LjM1OCAwIDAgMC02LjM3MS02LjM2JyBmaWxsPSclMjNGRkYnLyUzRSUzQy9nJTNFJTNDL3N2ZyUzRSUwQVwiKTtcbiAgYmFja2dyb3VuZC1yZXBlYXQ6IG5vLXJlcGVhdDtcbiAgYmFja2dyb3VuZC1wb3NpdGlvbjogY2VudGVyO1xuICBiYWNrZ3JvdW5kLXNpemU6IDEwMCU7XG4gIG92ZXJmbG93OiBoaWRkZW47XG4gIHRleHQtaW5kZW50OiAtOTAwMHB4O1xuICBwYWRkaW5nOiAwICFpbXBvcnRhbnQ7XG4gIHdpZHRoOiAxMDAlO1xuICBoZWlnaHQ6IDEwMCU7XG4gIGRpc3BsYXk6IGJsb2NrO1xufVxuIl19 */ \ No newline at end of file diff --git a/website/images/flags/en.svg b/website/images/flags/en.svg new file mode 100644 index 00000000000..b1dab887bcf --- /dev/null +++ b/website/images/flags/en.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/website/images/flags/es.svg b/website/images/flags/es.svg new file mode 100644 index 00000000000..ffecfa6e038 --- /dev/null +++ b/website/images/flags/es.svg @@ -0,0 +1,581 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/website/images/flags/fa.svg b/website/images/flags/fa.svg new file mode 100644 index 00000000000..35cd83dded2 --- /dev/null +++ b/website/images/flags/fa.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/website/images/flags/ja.svg b/website/images/flags/ja.svg new file mode 100644 index 00000000000..ad8286de8f8 --- /dev/null +++ b/website/images/flags/ja.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/website/images/flags/ru.svg b/website/images/flags/ru.svg new file mode 100644 index 00000000000..1733cc42728 --- /dev/null +++ b/website/images/flags/ru.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/website/images/flags/zh.svg b/website/images/flags/zh.svg new file mode 100644 index 00000000000..903b1eca640 --- /dev/null +++ b/website/images/flags/zh.svg @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/website/images/index/github.svg b/website/images/index/github.svg index 8f91c1d3402..0ca9ed095c7 100644 --- a/website/images/index/github.svg +++ b/website/images/index/github.svg @@ -1 +1 @@ - \ No newline at end of file + diff --git a/website/images/mkdocs/edit.svg b/website/images/mkdocs/edit.svg new file mode 100644 index 00000000000..e7db24a827b --- /dev/null +++ b/website/images/mkdocs/edit.svg @@ -0,0 +1,3 @@ + + + diff --git a/website/images/mkdocs/multi.svg b/website/images/mkdocs/multi.svg new file mode 100644 index 00000000000..f01b6f9e1b1 --- /dev/null +++ b/website/images/mkdocs/multi.svg @@ -0,0 +1,3 @@ + + + diff --git a/website/images/mkdocs/pdf.svg b/website/images/mkdocs/pdf.svg new file mode 100644 index 00000000000..8b082d47521 --- /dev/null +++ b/website/images/mkdocs/pdf.svg @@ -0,0 +1,3 @@ + + + diff --git a/website/images/mkdocs/search.svg b/website/images/mkdocs/search.svg new file mode 100644 index 00000000000..975ed8e89ee --- /dev/null +++ b/website/images/mkdocs/search.svg @@ -0,0 +1,3 @@ + + + diff --git a/website/images/mkdocs/single.svg b/website/images/mkdocs/single.svg new file mode 100644 index 00000000000..e6196bc1c29 --- /dev/null +++ b/website/images/mkdocs/single.svg @@ -0,0 +1,3 @@ + + + diff --git a/website/js/base.js b/website/js/base.js index 17f4c98b39b..8dfd426d6ed 100644 --- a/website/js/base.js +++ b/website/js/base.js @@ -1,30 +1,35 @@ -$(document).ready(function () { - var name = $('#logo-text').attr('alt').trim().toLowerCase(); - var feedback_address = name + '-feedback' + '@yandex-team.com'; - var feedback_email = $('#feedback_email'); - feedback_email.attr('href', 'mailto:' + feedback_address); - feedback_email.html(feedback_address); +(function () { + var logo_text = $('#logo-text'); + if (logo_text.length) { + var name = logo_text.attr('alt').trim().toLowerCase(); + var feedback_address = name + '-feedback' + '@yandex-team.com'; + var feedback_email = $('#feedback_email'); + feedback_email.attr('href', 'mailto:' + feedback_address); + feedback_email.html(feedback_address); + } $(document).click(function (event) { var target = $(event.target); var target_id = target.attr('id'); var selector = target.attr('href'); var is_tab = target.attr('role') === 'tab'; + var is_collapse = target.attr('data-toggle') === 'collapse'; var navbar_toggle = $('#navbar-toggle'); navbar_toggle.collapse('hide'); + $('.algolia-autocomplete .ds-dropdown-menu').hide(); if (target_id && target_id.startsWith('logo-')) { selector = '#'; } - if (selector && selector.startsWith('#') && !is_tab) { + if (selector && selector.startsWith('#') && !is_tab && !is_collapse) { event.preventDefault(); var dst = window.location.href.replace(window.location.hash, ''); var offset = 0; if (selector !== '#') { - offset = $(selector).offset().top - navbar_toggle.height() * 1.5; + offset = $(selector).offset().top - $('#top-nav').height() * 1.5; dst += selector; } $('html, body').animate({ @@ -57,6 +62,21 @@ $(document).ready(function () { d.addEventListener("DOMContentLoaded", f, false); } else { f(); } })(document, window, "yandex_metrika_callbacks2"); -}); - -{% include "js/index.js" %} + var beforePrint = function() { + var details = document.getElementsByTagName("details"); + for (var i = 0; i < details.length; ++i) { + details[i].open = 1; + } + }; + if (window.matchMedia) { + window.matchMedia('print').addListener(function(q) { + if (q.matches) { + beforePrint(); + } + }); + if (window.matchMedia('(prefers-reduced-motion: reduce)').matches) { + $.fx.off = true; + } + } + window.onbeforeprint = beforePrint; +})(); diff --git a/website/js/bootstrap.js b/website/js/bootstrap.js new file mode 100644 index 00000000000..f1e68d3190f --- /dev/null +++ b/website/js/bootstrap.js @@ -0,0 +1,4521 @@ +/*! + * Bootstrap v4.4.1 (https://getbootstrap.com/) + * Copyright 2011-2019 The Bootstrap Authors (https://github.com/twbs/bootstrap/graphs/contributors) + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ +(function (global, factory) { + typeof exports === 'object' && typeof module !== 'undefined' ? factory(exports, require('jquery'), require('popper.js')) : + typeof define === 'function' && define.amd ? define(['exports', 'jquery', 'popper.js'], factory) : + (global = global || self, factory(global.bootstrap = {}, global.jQuery, global.Popper)); +}(this, (function (exports, $, Popper) { 'use strict'; + + $ = $ && $.hasOwnProperty('default') ? $['default'] : $; + Popper = Popper && Popper.hasOwnProperty('default') ? Popper['default'] : Popper; + + function _defineProperties(target, props) { + for (var i = 0; i < props.length; i++) { + var descriptor = props[i]; + descriptor.enumerable = descriptor.enumerable || false; + descriptor.configurable = true; + if ("value" in descriptor) descriptor.writable = true; + Object.defineProperty(target, descriptor.key, descriptor); + } + } + + function _createClass(Constructor, protoProps, staticProps) { + if (protoProps) _defineProperties(Constructor.prototype, protoProps); + if (staticProps) _defineProperties(Constructor, staticProps); + return Constructor; + } + + function _defineProperty(obj, key, value) { + if (key in obj) { + Object.defineProperty(obj, key, { + value: value, + enumerable: true, + configurable: true, + writable: true + }); + } else { + obj[key] = value; + } + + return obj; + } + + function ownKeys(object, enumerableOnly) { + var keys = Object.keys(object); + + if (Object.getOwnPropertySymbols) { + var symbols = Object.getOwnPropertySymbols(object); + if (enumerableOnly) symbols = symbols.filter(function (sym) { + return Object.getOwnPropertyDescriptor(object, sym).enumerable; + }); + keys.push.apply(keys, symbols); + } + + return keys; + } + + function _objectSpread2(target) { + for (var i = 1; i < arguments.length; i++) { + var source = arguments[i] != null ? arguments[i] : {}; + + if (i % 2) { + ownKeys(Object(source), true).forEach(function (key) { + _defineProperty(target, key, source[key]); + }); + } else if (Object.getOwnPropertyDescriptors) { + Object.defineProperties(target, Object.getOwnPropertyDescriptors(source)); + } else { + ownKeys(Object(source)).forEach(function (key) { + Object.defineProperty(target, key, Object.getOwnPropertyDescriptor(source, key)); + }); + } + } + + return target; + } + + function _inheritsLoose(subClass, superClass) { + subClass.prototype = Object.create(superClass.prototype); + subClass.prototype.constructor = subClass; + subClass.__proto__ = superClass; + } + + /** + * -------------------------------------------------------------------------- + * Bootstrap (v4.4.1): util.js + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + * -------------------------------------------------------------------------- + */ + /** + * ------------------------------------------------------------------------ + * Private TransitionEnd Helpers + * ------------------------------------------------------------------------ + */ + + var TRANSITION_END = 'transitionend'; + var MAX_UID = 1000000; + var MILLISECONDS_MULTIPLIER = 1000; // Shoutout AngusCroll (https://goo.gl/pxwQGp) + + function toType(obj) { + return {}.toString.call(obj).match(/\s([a-z]+)/i)[1].toLowerCase(); + } + + function getSpecialTransitionEndEvent() { + return { + bindType: TRANSITION_END, + delegateType: TRANSITION_END, + handle: function handle(event) { + if ($(event.target).is(this)) { + return event.handleObj.handler.apply(this, arguments); // eslint-disable-line prefer-rest-params + } + + return undefined; // eslint-disable-line no-undefined + } + }; + } + + function transitionEndEmulator(duration) { + var _this = this; + + var called = false; + $(this).one(Util.TRANSITION_END, function () { + called = true; + }); + setTimeout(function () { + if (!called) { + Util.triggerTransitionEnd(_this); + } + }, duration); + return this; + } + + function setTransitionEndSupport() { + $.fn.emulateTransitionEnd = transitionEndEmulator; + $.event.special[Util.TRANSITION_END] = getSpecialTransitionEndEvent(); + } + /** + * -------------------------------------------------------------------------- + * Public Util Api + * -------------------------------------------------------------------------- + */ + + + var Util = { + TRANSITION_END: 'bsTransitionEnd', + getUID: function getUID(prefix) { + do { + // eslint-disable-next-line no-bitwise + prefix += ~~(Math.random() * MAX_UID); // "~~" acts like a faster Math.floor() here + } while (document.getElementById(prefix)); + + return prefix; + }, + getSelectorFromElement: function getSelectorFromElement(element) { + var selector = element.getAttribute('data-target'); + + if (!selector || selector === '#') { + var hrefAttr = element.getAttribute('href'); + selector = hrefAttr && hrefAttr !== '#' ? hrefAttr.trim() : ''; + } + + try { + return document.querySelector(selector) ? selector : null; + } catch (err) { + return null; + } + }, + getTransitionDurationFromElement: function getTransitionDurationFromElement(element) { + if (!element) { + return 0; + } // Get transition-duration of the element + + + var transitionDuration = $(element).css('transition-duration'); + var transitionDelay = $(element).css('transition-delay'); + var floatTransitionDuration = parseFloat(transitionDuration); + var floatTransitionDelay = parseFloat(transitionDelay); // Return 0 if element or transition duration is not found + + if (!floatTransitionDuration && !floatTransitionDelay) { + return 0; + } // If multiple durations are defined, take the first + + + transitionDuration = transitionDuration.split(',')[0]; + transitionDelay = transitionDelay.split(',')[0]; + return (parseFloat(transitionDuration) + parseFloat(transitionDelay)) * MILLISECONDS_MULTIPLIER; + }, + reflow: function reflow(element) { + return element.offsetHeight; + }, + triggerTransitionEnd: function triggerTransitionEnd(element) { + $(element).trigger(TRANSITION_END); + }, + // TODO: Remove in v5 + supportsTransitionEnd: function supportsTransitionEnd() { + return Boolean(TRANSITION_END); + }, + isElement: function isElement(obj) { + return (obj[0] || obj).nodeType; + }, + typeCheckConfig: function typeCheckConfig(componentName, config, configTypes) { + for (var property in configTypes) { + if (Object.prototype.hasOwnProperty.call(configTypes, property)) { + var expectedTypes = configTypes[property]; + var value = config[property]; + var valueType = value && Util.isElement(value) ? 'element' : toType(value); + + if (!new RegExp(expectedTypes).test(valueType)) { + throw new Error(componentName.toUpperCase() + ": " + ("Option \"" + property + "\" provided type \"" + valueType + "\" ") + ("but expected type \"" + expectedTypes + "\".")); + } + } + } + }, + findShadowRoot: function findShadowRoot(element) { + if (!document.documentElement.attachShadow) { + return null; + } // Can find the shadow root otherwise it'll return the document + + + if (typeof element.getRootNode === 'function') { + var root = element.getRootNode(); + return root instanceof ShadowRoot ? root : null; + } + + if (element instanceof ShadowRoot) { + return element; + } // when we don't find a shadow root + + + if (!element.parentNode) { + return null; + } + + return Util.findShadowRoot(element.parentNode); + }, + jQueryDetection: function jQueryDetection() { + if (typeof $ === 'undefined') { + throw new TypeError('Bootstrap\'s JavaScript requires jQuery. jQuery must be included before Bootstrap\'s JavaScript.'); + } + + var version = $.fn.jquery.split(' ')[0].split('.'); + var minMajor = 1; + var ltMajor = 2; + var minMinor = 9; + var minPatch = 1; + var maxMajor = 4; + + if (version[0] < ltMajor && version[1] < minMinor || version[0] === minMajor && version[1] === minMinor && version[2] < minPatch || version[0] >= maxMajor) { + throw new Error('Bootstrap\'s JavaScript requires at least jQuery v1.9.1 but less than v4.0.0'); + } + } + }; + Util.jQueryDetection(); + setTransitionEndSupport(); + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME = 'alert'; + var VERSION = '4.4.1'; + var DATA_KEY = 'bs.alert'; + var EVENT_KEY = "." + DATA_KEY; + var DATA_API_KEY = '.data-api'; + var JQUERY_NO_CONFLICT = $.fn[NAME]; + var Selector = { + DISMISS: '[data-dismiss="alert"]' + }; + var Event = { + CLOSE: "close" + EVENT_KEY, + CLOSED: "closed" + EVENT_KEY, + CLICK_DATA_API: "click" + EVENT_KEY + DATA_API_KEY + }; + var ClassName = { + ALERT: 'alert', + FADE: 'fade', + SHOW: 'show' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Alert = + /*#__PURE__*/ + function () { + function Alert(element) { + this._element = element; + } // Getters + + + var _proto = Alert.prototype; + + // Public + _proto.close = function close(element) { + var rootElement = this._element; + + if (element) { + rootElement = this._getRootElement(element); + } + + var customEvent = this._triggerCloseEvent(rootElement); + + if (customEvent.isDefaultPrevented()) { + return; + } + + this._removeElement(rootElement); + }; + + _proto.dispose = function dispose() { + $.removeData(this._element, DATA_KEY); + this._element = null; + } // Private + ; + + _proto._getRootElement = function _getRootElement(element) { + var selector = Util.getSelectorFromElement(element); + var parent = false; + + if (selector) { + parent = document.querySelector(selector); + } + + if (!parent) { + parent = $(element).closest("." + ClassName.ALERT)[0]; + } + + return parent; + }; + + _proto._triggerCloseEvent = function _triggerCloseEvent(element) { + var closeEvent = $.Event(Event.CLOSE); + $(element).trigger(closeEvent); + return closeEvent; + }; + + _proto._removeElement = function _removeElement(element) { + var _this = this; + + $(element).removeClass(ClassName.SHOW); + + if (!$(element).hasClass(ClassName.FADE)) { + this._destroyElement(element); + + return; + } + + var transitionDuration = Util.getTransitionDurationFromElement(element); + $(element).one(Util.TRANSITION_END, function (event) { + return _this._destroyElement(element, event); + }).emulateTransitionEnd(transitionDuration); + }; + + _proto._destroyElement = function _destroyElement(element) { + $(element).detach().trigger(Event.CLOSED).remove(); + } // Static + ; + + Alert._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var $element = $(this); + var data = $element.data(DATA_KEY); + + if (!data) { + data = new Alert(this); + $element.data(DATA_KEY, data); + } + + if (config === 'close') { + data[config](this); + } + }); + }; + + Alert._handleDismiss = function _handleDismiss(alertInstance) { + return function (event) { + if (event) { + event.preventDefault(); + } + + alertInstance.close(this); + }; + }; + + _createClass(Alert, null, [{ + key: "VERSION", + get: function get() { + return VERSION; + } + }]); + + return Alert; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event.CLICK_DATA_API, Selector.DISMISS, Alert._handleDismiss(new Alert())); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME] = Alert._jQueryInterface; + $.fn[NAME].Constructor = Alert; + + $.fn[NAME].noConflict = function () { + $.fn[NAME] = JQUERY_NO_CONFLICT; + return Alert._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$1 = 'button'; + var VERSION$1 = '4.4.1'; + var DATA_KEY$1 = 'bs.button'; + var EVENT_KEY$1 = "." + DATA_KEY$1; + var DATA_API_KEY$1 = '.data-api'; + var JQUERY_NO_CONFLICT$1 = $.fn[NAME$1]; + var ClassName$1 = { + ACTIVE: 'active', + BUTTON: 'btn', + FOCUS: 'focus' + }; + var Selector$1 = { + DATA_TOGGLE_CARROT: '[data-toggle^="button"]', + DATA_TOGGLES: '[data-toggle="buttons"]', + DATA_TOGGLE: '[data-toggle="button"]', + DATA_TOGGLES_BUTTONS: '[data-toggle="buttons"] .btn', + INPUT: 'input:not([type="hidden"])', + ACTIVE: '.active', + BUTTON: '.btn' + }; + var Event$1 = { + CLICK_DATA_API: "click" + EVENT_KEY$1 + DATA_API_KEY$1, + FOCUS_BLUR_DATA_API: "focus" + EVENT_KEY$1 + DATA_API_KEY$1 + " " + ("blur" + EVENT_KEY$1 + DATA_API_KEY$1), + LOAD_DATA_API: "load" + EVENT_KEY$1 + DATA_API_KEY$1 + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Button = + /*#__PURE__*/ + function () { + function Button(element) { + this._element = element; + } // Getters + + + var _proto = Button.prototype; + + // Public + _proto.toggle = function toggle() { + var triggerChangeEvent = true; + var addAriaPressed = true; + var rootElement = $(this._element).closest(Selector$1.DATA_TOGGLES)[0]; + + if (rootElement) { + var input = this._element.querySelector(Selector$1.INPUT); + + if (input) { + if (input.type === 'radio') { + if (input.checked && this._element.classList.contains(ClassName$1.ACTIVE)) { + triggerChangeEvent = false; + } else { + var activeElement = rootElement.querySelector(Selector$1.ACTIVE); + + if (activeElement) { + $(activeElement).removeClass(ClassName$1.ACTIVE); + } + } + } else if (input.type === 'checkbox') { + if (this._element.tagName === 'LABEL' && input.checked === this._element.classList.contains(ClassName$1.ACTIVE)) { + triggerChangeEvent = false; + } + } else { + // if it's not a radio button or checkbox don't add a pointless/invalid checked property to the input + triggerChangeEvent = false; + } + + if (triggerChangeEvent) { + input.checked = !this._element.classList.contains(ClassName$1.ACTIVE); + $(input).trigger('change'); + } + + input.focus(); + addAriaPressed = false; + } + } + + if (!(this._element.hasAttribute('disabled') || this._element.classList.contains('disabled'))) { + if (addAriaPressed) { + this._element.setAttribute('aria-pressed', !this._element.classList.contains(ClassName$1.ACTIVE)); + } + + if (triggerChangeEvent) { + $(this._element).toggleClass(ClassName$1.ACTIVE); + } + } + }; + + _proto.dispose = function dispose() { + $.removeData(this._element, DATA_KEY$1); + this._element = null; + } // Static + ; + + Button._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var data = $(this).data(DATA_KEY$1); + + if (!data) { + data = new Button(this); + $(this).data(DATA_KEY$1, data); + } + + if (config === 'toggle') { + data[config](); + } + }); + }; + + _createClass(Button, null, [{ + key: "VERSION", + get: function get() { + return VERSION$1; + } + }]); + + return Button; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event$1.CLICK_DATA_API, Selector$1.DATA_TOGGLE_CARROT, function (event) { + var button = event.target; + + if (!$(button).hasClass(ClassName$1.BUTTON)) { + button = $(button).closest(Selector$1.BUTTON)[0]; + } + + if (!button || button.hasAttribute('disabled') || button.classList.contains('disabled')) { + event.preventDefault(); // work around Firefox bug #1540995 + } else { + var inputBtn = button.querySelector(Selector$1.INPUT); + + if (inputBtn && (inputBtn.hasAttribute('disabled') || inputBtn.classList.contains('disabled'))) { + event.preventDefault(); // work around Firefox bug #1540995 + + return; + } + + Button._jQueryInterface.call($(button), 'toggle'); + } + }).on(Event$1.FOCUS_BLUR_DATA_API, Selector$1.DATA_TOGGLE_CARROT, function (event) { + var button = $(event.target).closest(Selector$1.BUTTON)[0]; + $(button).toggleClass(ClassName$1.FOCUS, /^focus(in)?$/.test(event.type)); + }); + $(window).on(Event$1.LOAD_DATA_API, function () { + // ensure correct active class is set to match the controls' actual values/states + // find all checkboxes/readio buttons inside data-toggle groups + var buttons = [].slice.call(document.querySelectorAll(Selector$1.DATA_TOGGLES_BUTTONS)); + + for (var i = 0, len = buttons.length; i < len; i++) { + var button = buttons[i]; + var input = button.querySelector(Selector$1.INPUT); + + if (input.checked || input.hasAttribute('checked')) { + button.classList.add(ClassName$1.ACTIVE); + } else { + button.classList.remove(ClassName$1.ACTIVE); + } + } // find all button toggles + + + buttons = [].slice.call(document.querySelectorAll(Selector$1.DATA_TOGGLE)); + + for (var _i = 0, _len = buttons.length; _i < _len; _i++) { + var _button = buttons[_i]; + + if (_button.getAttribute('aria-pressed') === 'true') { + _button.classList.add(ClassName$1.ACTIVE); + } else { + _button.classList.remove(ClassName$1.ACTIVE); + } + } + }); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME$1] = Button._jQueryInterface; + $.fn[NAME$1].Constructor = Button; + + $.fn[NAME$1].noConflict = function () { + $.fn[NAME$1] = JQUERY_NO_CONFLICT$1; + return Button._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$2 = 'carousel'; + var VERSION$2 = '4.4.1'; + var DATA_KEY$2 = 'bs.carousel'; + var EVENT_KEY$2 = "." + DATA_KEY$2; + var DATA_API_KEY$2 = '.data-api'; + var JQUERY_NO_CONFLICT$2 = $.fn[NAME$2]; + var ARROW_LEFT_KEYCODE = 37; // KeyboardEvent.which value for left arrow key + + var ARROW_RIGHT_KEYCODE = 39; // KeyboardEvent.which value for right arrow key + + var TOUCHEVENT_COMPAT_WAIT = 500; // Time for mouse compat events to fire after touch + + var SWIPE_THRESHOLD = 40; + var Default = { + interval: 5000, + keyboard: true, + slide: false, + pause: 'hover', + wrap: true, + touch: true + }; + var DefaultType = { + interval: '(number|boolean)', + keyboard: 'boolean', + slide: '(boolean|string)', + pause: '(string|boolean)', + wrap: 'boolean', + touch: 'boolean' + }; + var Direction = { + NEXT: 'next', + PREV: 'prev', + LEFT: 'left', + RIGHT: 'right' + }; + var Event$2 = { + SLIDE: "slide" + EVENT_KEY$2, + SLID: "slid" + EVENT_KEY$2, + KEYDOWN: "keydown" + EVENT_KEY$2, + MOUSEENTER: "mouseenter" + EVENT_KEY$2, + MOUSELEAVE: "mouseleave" + EVENT_KEY$2, + TOUCHSTART: "touchstart" + EVENT_KEY$2, + TOUCHMOVE: "touchmove" + EVENT_KEY$2, + TOUCHEND: "touchend" + EVENT_KEY$2, + POINTERDOWN: "pointerdown" + EVENT_KEY$2, + POINTERUP: "pointerup" + EVENT_KEY$2, + DRAG_START: "dragstart" + EVENT_KEY$2, + LOAD_DATA_API: "load" + EVENT_KEY$2 + DATA_API_KEY$2, + CLICK_DATA_API: "click" + EVENT_KEY$2 + DATA_API_KEY$2 + }; + var ClassName$2 = { + CAROUSEL: 'carousel', + ACTIVE: 'active', + SLIDE: 'slide', + RIGHT: 'carousel-item-right', + LEFT: 'carousel-item-left', + NEXT: 'carousel-item-next', + PREV: 'carousel-item-prev', + ITEM: 'carousel-item', + POINTER_EVENT: 'pointer-event' + }; + var Selector$2 = { + ACTIVE: '.active', + ACTIVE_ITEM: '.active.carousel-item', + ITEM: '.carousel-item', + ITEM_IMG: '.carousel-item img', + NEXT_PREV: '.carousel-item-next, .carousel-item-prev', + INDICATORS: '.carousel-indicators', + DATA_SLIDE: '[data-slide], [data-slide-to]', + DATA_RIDE: '[data-ride="carousel"]' + }; + var PointerType = { + TOUCH: 'touch', + PEN: 'pen' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Carousel = + /*#__PURE__*/ + function () { + function Carousel(element, config) { + this._items = null; + this._interval = null; + this._activeElement = null; + this._isPaused = false; + this._isSliding = false; + this.touchTimeout = null; + this.touchStartX = 0; + this.touchDeltaX = 0; + this._config = this._getConfig(config); + this._element = element; + this._indicatorsElement = this._element.querySelector(Selector$2.INDICATORS); + this._touchSupported = 'ontouchstart' in document.documentElement || navigator.maxTouchPoints > 0; + this._pointerEvent = Boolean(window.PointerEvent || window.MSPointerEvent); + + this._addEventListeners(); + } // Getters + + + var _proto = Carousel.prototype; + + // Public + _proto.next = function next() { + if (!this._isSliding) { + this._slide(Direction.NEXT); + } + }; + + _proto.nextWhenVisible = function nextWhenVisible() { + // Don't call next when the page isn't visible + // or the carousel or its parent isn't visible + if (!document.hidden && $(this._element).is(':visible') && $(this._element).css('visibility') !== 'hidden') { + this.next(); + } + }; + + _proto.prev = function prev() { + if (!this._isSliding) { + this._slide(Direction.PREV); + } + }; + + _proto.pause = function pause(event) { + if (!event) { + this._isPaused = true; + } + + if (this._element.querySelector(Selector$2.NEXT_PREV)) { + Util.triggerTransitionEnd(this._element); + this.cycle(true); + } + + clearInterval(this._interval); + this._interval = null; + }; + + _proto.cycle = function cycle(event) { + if (!event) { + this._isPaused = false; + } + + if (this._interval) { + clearInterval(this._interval); + this._interval = null; + } + + if (this._config.interval && !this._isPaused) { + this._interval = setInterval((document.visibilityState ? this.nextWhenVisible : this.next).bind(this), this._config.interval); + } + }; + + _proto.to = function to(index) { + var _this = this; + + this._activeElement = this._element.querySelector(Selector$2.ACTIVE_ITEM); + + var activeIndex = this._getItemIndex(this._activeElement); + + if (index > this._items.length - 1 || index < 0) { + return; + } + + if (this._isSliding) { + $(this._element).one(Event$2.SLID, function () { + return _this.to(index); + }); + return; + } + + if (activeIndex === index) { + this.pause(); + this.cycle(); + return; + } + + var direction = index > activeIndex ? Direction.NEXT : Direction.PREV; + + this._slide(direction, this._items[index]); + }; + + _proto.dispose = function dispose() { + $(this._element).off(EVENT_KEY$2); + $.removeData(this._element, DATA_KEY$2); + this._items = null; + this._config = null; + this._element = null; + this._interval = null; + this._isPaused = null; + this._isSliding = null; + this._activeElement = null; + this._indicatorsElement = null; + } // Private + ; + + _proto._getConfig = function _getConfig(config) { + config = _objectSpread2({}, Default, {}, config); + Util.typeCheckConfig(NAME$2, config, DefaultType); + return config; + }; + + _proto._handleSwipe = function _handleSwipe() { + var absDeltax = Math.abs(this.touchDeltaX); + + if (absDeltax <= SWIPE_THRESHOLD) { + return; + } + + var direction = absDeltax / this.touchDeltaX; + this.touchDeltaX = 0; // swipe left + + if (direction > 0) { + this.prev(); + } // swipe right + + + if (direction < 0) { + this.next(); + } + }; + + _proto._addEventListeners = function _addEventListeners() { + var _this2 = this; + + if (this._config.keyboard) { + $(this._element).on(Event$2.KEYDOWN, function (event) { + return _this2._keydown(event); + }); + } + + if (this._config.pause === 'hover') { + $(this._element).on(Event$2.MOUSEENTER, function (event) { + return _this2.pause(event); + }).on(Event$2.MOUSELEAVE, function (event) { + return _this2.cycle(event); + }); + } + + if (this._config.touch) { + this._addTouchEventListeners(); + } + }; + + _proto._addTouchEventListeners = function _addTouchEventListeners() { + var _this3 = this; + + if (!this._touchSupported) { + return; + } + + var start = function start(event) { + if (_this3._pointerEvent && PointerType[event.originalEvent.pointerType.toUpperCase()]) { + _this3.touchStartX = event.originalEvent.clientX; + } else if (!_this3._pointerEvent) { + _this3.touchStartX = event.originalEvent.touches[0].clientX; + } + }; + + var move = function move(event) { + // ensure swiping with one touch and not pinching + if (event.originalEvent.touches && event.originalEvent.touches.length > 1) { + _this3.touchDeltaX = 0; + } else { + _this3.touchDeltaX = event.originalEvent.touches[0].clientX - _this3.touchStartX; + } + }; + + var end = function end(event) { + if (_this3._pointerEvent && PointerType[event.originalEvent.pointerType.toUpperCase()]) { + _this3.touchDeltaX = event.originalEvent.clientX - _this3.touchStartX; + } + + _this3._handleSwipe(); + + if (_this3._config.pause === 'hover') { + // If it's a touch-enabled device, mouseenter/leave are fired as + // part of the mouse compatibility events on first tap - the carousel + // would stop cycling until user tapped out of it; + // here, we listen for touchend, explicitly pause the carousel + // (as if it's the second time we tap on it, mouseenter compat event + // is NOT fired) and after a timeout (to allow for mouse compatibility + // events to fire) we explicitly restart cycling + _this3.pause(); + + if (_this3.touchTimeout) { + clearTimeout(_this3.touchTimeout); + } + + _this3.touchTimeout = setTimeout(function (event) { + return _this3.cycle(event); + }, TOUCHEVENT_COMPAT_WAIT + _this3._config.interval); + } + }; + + $(this._element.querySelectorAll(Selector$2.ITEM_IMG)).on(Event$2.DRAG_START, function (e) { + return e.preventDefault(); + }); + + if (this._pointerEvent) { + $(this._element).on(Event$2.POINTERDOWN, function (event) { + return start(event); + }); + $(this._element).on(Event$2.POINTERUP, function (event) { + return end(event); + }); + + this._element.classList.add(ClassName$2.POINTER_EVENT); + } else { + $(this._element).on(Event$2.TOUCHSTART, function (event) { + return start(event); + }); + $(this._element).on(Event$2.TOUCHMOVE, function (event) { + return move(event); + }); + $(this._element).on(Event$2.TOUCHEND, function (event) { + return end(event); + }); + } + }; + + _proto._keydown = function _keydown(event) { + if (/input|textarea/i.test(event.target.tagName)) { + return; + } + + switch (event.which) { + case ARROW_LEFT_KEYCODE: + event.preventDefault(); + this.prev(); + break; + + case ARROW_RIGHT_KEYCODE: + event.preventDefault(); + this.next(); + break; + } + }; + + _proto._getItemIndex = function _getItemIndex(element) { + this._items = element && element.parentNode ? [].slice.call(element.parentNode.querySelectorAll(Selector$2.ITEM)) : []; + return this._items.indexOf(element); + }; + + _proto._getItemByDirection = function _getItemByDirection(direction, activeElement) { + var isNextDirection = direction === Direction.NEXT; + var isPrevDirection = direction === Direction.PREV; + + var activeIndex = this._getItemIndex(activeElement); + + var lastItemIndex = this._items.length - 1; + var isGoingToWrap = isPrevDirection && activeIndex === 0 || isNextDirection && activeIndex === lastItemIndex; + + if (isGoingToWrap && !this._config.wrap) { + return activeElement; + } + + var delta = direction === Direction.PREV ? -1 : 1; + var itemIndex = (activeIndex + delta) % this._items.length; + return itemIndex === -1 ? this._items[this._items.length - 1] : this._items[itemIndex]; + }; + + _proto._triggerSlideEvent = function _triggerSlideEvent(relatedTarget, eventDirectionName) { + var targetIndex = this._getItemIndex(relatedTarget); + + var fromIndex = this._getItemIndex(this._element.querySelector(Selector$2.ACTIVE_ITEM)); + + var slideEvent = $.Event(Event$2.SLIDE, { + relatedTarget: relatedTarget, + direction: eventDirectionName, + from: fromIndex, + to: targetIndex + }); + $(this._element).trigger(slideEvent); + return slideEvent; + }; + + _proto._setActiveIndicatorElement = function _setActiveIndicatorElement(element) { + if (this._indicatorsElement) { + var indicators = [].slice.call(this._indicatorsElement.querySelectorAll(Selector$2.ACTIVE)); + $(indicators).removeClass(ClassName$2.ACTIVE); + + var nextIndicator = this._indicatorsElement.children[this._getItemIndex(element)]; + + if (nextIndicator) { + $(nextIndicator).addClass(ClassName$2.ACTIVE); + } + } + }; + + _proto._slide = function _slide(direction, element) { + var _this4 = this; + + var activeElement = this._element.querySelector(Selector$2.ACTIVE_ITEM); + + var activeElementIndex = this._getItemIndex(activeElement); + + var nextElement = element || activeElement && this._getItemByDirection(direction, activeElement); + + var nextElementIndex = this._getItemIndex(nextElement); + + var isCycling = Boolean(this._interval); + var directionalClassName; + var orderClassName; + var eventDirectionName; + + if (direction === Direction.NEXT) { + directionalClassName = ClassName$2.LEFT; + orderClassName = ClassName$2.NEXT; + eventDirectionName = Direction.LEFT; + } else { + directionalClassName = ClassName$2.RIGHT; + orderClassName = ClassName$2.PREV; + eventDirectionName = Direction.RIGHT; + } + + if (nextElement && $(nextElement).hasClass(ClassName$2.ACTIVE)) { + this._isSliding = false; + return; + } + + var slideEvent = this._triggerSlideEvent(nextElement, eventDirectionName); + + if (slideEvent.isDefaultPrevented()) { + return; + } + + if (!activeElement || !nextElement) { + // Some weirdness is happening, so we bail + return; + } + + this._isSliding = true; + + if (isCycling) { + this.pause(); + } + + this._setActiveIndicatorElement(nextElement); + + var slidEvent = $.Event(Event$2.SLID, { + relatedTarget: nextElement, + direction: eventDirectionName, + from: activeElementIndex, + to: nextElementIndex + }); + + if ($(this._element).hasClass(ClassName$2.SLIDE)) { + $(nextElement).addClass(orderClassName); + Util.reflow(nextElement); + $(activeElement).addClass(directionalClassName); + $(nextElement).addClass(directionalClassName); + var nextElementInterval = parseInt(nextElement.getAttribute('data-interval'), 10); + + if (nextElementInterval) { + this._config.defaultInterval = this._config.defaultInterval || this._config.interval; + this._config.interval = nextElementInterval; + } else { + this._config.interval = this._config.defaultInterval || this._config.interval; + } + + var transitionDuration = Util.getTransitionDurationFromElement(activeElement); + $(activeElement).one(Util.TRANSITION_END, function () { + $(nextElement).removeClass(directionalClassName + " " + orderClassName).addClass(ClassName$2.ACTIVE); + $(activeElement).removeClass(ClassName$2.ACTIVE + " " + orderClassName + " " + directionalClassName); + _this4._isSliding = false; + setTimeout(function () { + return $(_this4._element).trigger(slidEvent); + }, 0); + }).emulateTransitionEnd(transitionDuration); + } else { + $(activeElement).removeClass(ClassName$2.ACTIVE); + $(nextElement).addClass(ClassName$2.ACTIVE); + this._isSliding = false; + $(this._element).trigger(slidEvent); + } + + if (isCycling) { + this.cycle(); + } + } // Static + ; + + Carousel._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var data = $(this).data(DATA_KEY$2); + + var _config = _objectSpread2({}, Default, {}, $(this).data()); + + if (typeof config === 'object') { + _config = _objectSpread2({}, _config, {}, config); + } + + var action = typeof config === 'string' ? config : _config.slide; + + if (!data) { + data = new Carousel(this, _config); + $(this).data(DATA_KEY$2, data); + } + + if (typeof config === 'number') { + data.to(config); + } else if (typeof action === 'string') { + if (typeof data[action] === 'undefined') { + throw new TypeError("No method named \"" + action + "\""); + } + + data[action](); + } else if (_config.interval && _config.ride) { + data.pause(); + data.cycle(); + } + }); + }; + + Carousel._dataApiClickHandler = function _dataApiClickHandler(event) { + var selector = Util.getSelectorFromElement(this); + + if (!selector) { + return; + } + + var target = $(selector)[0]; + + if (!target || !$(target).hasClass(ClassName$2.CAROUSEL)) { + return; + } + + var config = _objectSpread2({}, $(target).data(), {}, $(this).data()); + + var slideIndex = this.getAttribute('data-slide-to'); + + if (slideIndex) { + config.interval = false; + } + + Carousel._jQueryInterface.call($(target), config); + + if (slideIndex) { + $(target).data(DATA_KEY$2).to(slideIndex); + } + + event.preventDefault(); + }; + + _createClass(Carousel, null, [{ + key: "VERSION", + get: function get() { + return VERSION$2; + } + }, { + key: "Default", + get: function get() { + return Default; + } + }]); + + return Carousel; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event$2.CLICK_DATA_API, Selector$2.DATA_SLIDE, Carousel._dataApiClickHandler); + $(window).on(Event$2.LOAD_DATA_API, function () { + var carousels = [].slice.call(document.querySelectorAll(Selector$2.DATA_RIDE)); + + for (var i = 0, len = carousels.length; i < len; i++) { + var $carousel = $(carousels[i]); + + Carousel._jQueryInterface.call($carousel, $carousel.data()); + } + }); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME$2] = Carousel._jQueryInterface; + $.fn[NAME$2].Constructor = Carousel; + + $.fn[NAME$2].noConflict = function () { + $.fn[NAME$2] = JQUERY_NO_CONFLICT$2; + return Carousel._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$3 = 'collapse'; + var VERSION$3 = '4.4.1'; + var DATA_KEY$3 = 'bs.collapse'; + var EVENT_KEY$3 = "." + DATA_KEY$3; + var DATA_API_KEY$3 = '.data-api'; + var JQUERY_NO_CONFLICT$3 = $.fn[NAME$3]; + var Default$1 = { + toggle: true, + parent: '' + }; + var DefaultType$1 = { + toggle: 'boolean', + parent: '(string|element)' + }; + var Event$3 = { + SHOW: "show" + EVENT_KEY$3, + SHOWN: "shown" + EVENT_KEY$3, + HIDE: "hide" + EVENT_KEY$3, + HIDDEN: "hidden" + EVENT_KEY$3, + CLICK_DATA_API: "click" + EVENT_KEY$3 + DATA_API_KEY$3 + }; + var ClassName$3 = { + SHOW: 'show', + COLLAPSE: 'collapse', + COLLAPSING: 'collapsing', + COLLAPSED: 'collapsed' + }; + var Dimension = { + WIDTH: 'width', + HEIGHT: 'height' + }; + var Selector$3 = { + ACTIVES: '.show, .collapsing', + DATA_TOGGLE: '[data-toggle="collapse"]' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Collapse = + /*#__PURE__*/ + function () { + function Collapse(element, config) { + this._isTransitioning = false; + this._element = element; + this._config = this._getConfig(config); + this._triggerArray = [].slice.call(document.querySelectorAll("[data-toggle=\"collapse\"][href=\"#" + element.id + "\"]," + ("[data-toggle=\"collapse\"][data-target=\"#" + element.id + "\"]"))); + var toggleList = [].slice.call(document.querySelectorAll(Selector$3.DATA_TOGGLE)); + + for (var i = 0, len = toggleList.length; i < len; i++) { + var elem = toggleList[i]; + var selector = Util.getSelectorFromElement(elem); + var filterElement = [].slice.call(document.querySelectorAll(selector)).filter(function (foundElem) { + return foundElem === element; + }); + + if (selector !== null && filterElement.length > 0) { + this._selector = selector; + + this._triggerArray.push(elem); + } + } + + this._parent = this._config.parent ? this._getParent() : null; + + if (!this._config.parent) { + this._addAriaAndCollapsedClass(this._element, this._triggerArray); + } + + if (this._config.toggle) { + this.toggle(); + } + } // Getters + + + var _proto = Collapse.prototype; + + // Public + _proto.toggle = function toggle() { + if ($(this._element).hasClass(ClassName$3.SHOW)) { + this.hide(); + } else { + this.show(); + } + }; + + _proto.show = function show() { + var _this = this; + + if (this._isTransitioning || $(this._element).hasClass(ClassName$3.SHOW)) { + return; + } + + var actives; + var activesData; + + if (this._parent) { + actives = [].slice.call(this._parent.querySelectorAll(Selector$3.ACTIVES)).filter(function (elem) { + if (typeof _this._config.parent === 'string') { + return elem.getAttribute('data-parent') === _this._config.parent; + } + + return elem.classList.contains(ClassName$3.COLLAPSE); + }); + + if (actives.length === 0) { + actives = null; + } + } + + if (actives) { + activesData = $(actives).not(this._selector).data(DATA_KEY$3); + + if (activesData && activesData._isTransitioning) { + return; + } + } + + var startEvent = $.Event(Event$3.SHOW); + $(this._element).trigger(startEvent); + + if (startEvent.isDefaultPrevented()) { + return; + } + + if (actives) { + Collapse._jQueryInterface.call($(actives).not(this._selector), 'hide'); + + if (!activesData) { + $(actives).data(DATA_KEY$3, null); + } + } + + var dimension = this._getDimension(); + + $(this._element).removeClass(ClassName$3.COLLAPSE).addClass(ClassName$3.COLLAPSING); + this._element.style[dimension] = 0; + + if (this._triggerArray.length) { + $(this._triggerArray).removeClass(ClassName$3.COLLAPSED).attr('aria-expanded', true); + } + + this.setTransitioning(true); + + var complete = function complete() { + $(_this._element).removeClass(ClassName$3.COLLAPSING).addClass(ClassName$3.COLLAPSE).addClass(ClassName$3.SHOW); + _this._element.style[dimension] = ''; + + _this.setTransitioning(false); + + $(_this._element).trigger(Event$3.SHOWN); + }; + + var capitalizedDimension = dimension[0].toUpperCase() + dimension.slice(1); + var scrollSize = "scroll" + capitalizedDimension; + var transitionDuration = Util.getTransitionDurationFromElement(this._element); + $(this._element).one(Util.TRANSITION_END, complete).emulateTransitionEnd(transitionDuration); + this._element.style[dimension] = this._element[scrollSize] + "px"; + }; + + _proto.hide = function hide() { + var _this2 = this; + + if (this._isTransitioning || !$(this._element).hasClass(ClassName$3.SHOW)) { + return; + } + + var startEvent = $.Event(Event$3.HIDE); + $(this._element).trigger(startEvent); + + if (startEvent.isDefaultPrevented()) { + return; + } + + var dimension = this._getDimension(); + + this._element.style[dimension] = this._element.getBoundingClientRect()[dimension] + "px"; + Util.reflow(this._element); + $(this._element).addClass(ClassName$3.COLLAPSING).removeClass(ClassName$3.COLLAPSE).removeClass(ClassName$3.SHOW); + var triggerArrayLength = this._triggerArray.length; + + if (triggerArrayLength > 0) { + for (var i = 0; i < triggerArrayLength; i++) { + var trigger = this._triggerArray[i]; + var selector = Util.getSelectorFromElement(trigger); + + if (selector !== null) { + var $elem = $([].slice.call(document.querySelectorAll(selector))); + + if (!$elem.hasClass(ClassName$3.SHOW)) { + $(trigger).addClass(ClassName$3.COLLAPSED).attr('aria-expanded', false); + } + } + } + } + + this.setTransitioning(true); + + var complete = function complete() { + _this2.setTransitioning(false); + + $(_this2._element).removeClass(ClassName$3.COLLAPSING).addClass(ClassName$3.COLLAPSE).trigger(Event$3.HIDDEN); + }; + + this._element.style[dimension] = ''; + var transitionDuration = Util.getTransitionDurationFromElement(this._element); + $(this._element).one(Util.TRANSITION_END, complete).emulateTransitionEnd(transitionDuration); + }; + + _proto.setTransitioning = function setTransitioning(isTransitioning) { + this._isTransitioning = isTransitioning; + }; + + _proto.dispose = function dispose() { + $.removeData(this._element, DATA_KEY$3); + this._config = null; + this._parent = null; + this._element = null; + this._triggerArray = null; + this._isTransitioning = null; + } // Private + ; + + _proto._getConfig = function _getConfig(config) { + config = _objectSpread2({}, Default$1, {}, config); + config.toggle = Boolean(config.toggle); // Coerce string values + + Util.typeCheckConfig(NAME$3, config, DefaultType$1); + return config; + }; + + _proto._getDimension = function _getDimension() { + var hasWidth = $(this._element).hasClass(Dimension.WIDTH); + return hasWidth ? Dimension.WIDTH : Dimension.HEIGHT; + }; + + _proto._getParent = function _getParent() { + var _this3 = this; + + var parent; + + if (Util.isElement(this._config.parent)) { + parent = this._config.parent; // It's a jQuery object + + if (typeof this._config.parent.jquery !== 'undefined') { + parent = this._config.parent[0]; + } + } else { + parent = document.querySelector(this._config.parent); + } + + var selector = "[data-toggle=\"collapse\"][data-parent=\"" + this._config.parent + "\"]"; + var children = [].slice.call(parent.querySelectorAll(selector)); + $(children).each(function (i, element) { + _this3._addAriaAndCollapsedClass(Collapse._getTargetFromElement(element), [element]); + }); + return parent; + }; + + _proto._addAriaAndCollapsedClass = function _addAriaAndCollapsedClass(element, triggerArray) { + var isOpen = $(element).hasClass(ClassName$3.SHOW); + + if (triggerArray.length) { + $(triggerArray).toggleClass(ClassName$3.COLLAPSED, !isOpen).attr('aria-expanded', isOpen); + } + } // Static + ; + + Collapse._getTargetFromElement = function _getTargetFromElement(element) { + var selector = Util.getSelectorFromElement(element); + return selector ? document.querySelector(selector) : null; + }; + + Collapse._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var $this = $(this); + var data = $this.data(DATA_KEY$3); + + var _config = _objectSpread2({}, Default$1, {}, $this.data(), {}, typeof config === 'object' && config ? config : {}); + + if (!data && _config.toggle && /show|hide/.test(config)) { + _config.toggle = false; + } + + if (!data) { + data = new Collapse(this, _config); + $this.data(DATA_KEY$3, data); + } + + if (typeof config === 'string') { + if (typeof data[config] === 'undefined') { + throw new TypeError("No method named \"" + config + "\""); + } + + data[config](); + } + }); + }; + + _createClass(Collapse, null, [{ + key: "VERSION", + get: function get() { + return VERSION$3; + } + }, { + key: "Default", + get: function get() { + return Default$1; + } + }]); + + return Collapse; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event$3.CLICK_DATA_API, Selector$3.DATA_TOGGLE, function (event) { + // preventDefault only for elements (which change the URL) not inside the collapsible element + if (event.currentTarget.tagName === 'A') { + event.preventDefault(); + } + + var $trigger = $(this); + var selector = Util.getSelectorFromElement(this); + var selectors = [].slice.call(document.querySelectorAll(selector)); + $(selectors).each(function () { + var $target = $(this); + var data = $target.data(DATA_KEY$3); + var config = data ? 'toggle' : $trigger.data(); + + Collapse._jQueryInterface.call($target, config); + }); + }); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME$3] = Collapse._jQueryInterface; + $.fn[NAME$3].Constructor = Collapse; + + $.fn[NAME$3].noConflict = function () { + $.fn[NAME$3] = JQUERY_NO_CONFLICT$3; + return Collapse._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$4 = 'dropdown'; + var VERSION$4 = '4.4.1'; + var DATA_KEY$4 = 'bs.dropdown'; + var EVENT_KEY$4 = "." + DATA_KEY$4; + var DATA_API_KEY$4 = '.data-api'; + var JQUERY_NO_CONFLICT$4 = $.fn[NAME$4]; + var ESCAPE_KEYCODE = 27; // KeyboardEvent.which value for Escape (Esc) key + + var SPACE_KEYCODE = 32; // KeyboardEvent.which value for space key + + var TAB_KEYCODE = 9; // KeyboardEvent.which value for tab key + + var ARROW_UP_KEYCODE = 38; // KeyboardEvent.which value for up arrow key + + var ARROW_DOWN_KEYCODE = 40; // KeyboardEvent.which value for down arrow key + + var RIGHT_MOUSE_BUTTON_WHICH = 3; // MouseEvent.which value for the right button (assuming a right-handed mouse) + + var REGEXP_KEYDOWN = new RegExp(ARROW_UP_KEYCODE + "|" + ARROW_DOWN_KEYCODE + "|" + ESCAPE_KEYCODE); + var Event$4 = { + HIDE: "hide" + EVENT_KEY$4, + HIDDEN: "hidden" + EVENT_KEY$4, + SHOW: "show" + EVENT_KEY$4, + SHOWN: "shown" + EVENT_KEY$4, + CLICK: "click" + EVENT_KEY$4, + CLICK_DATA_API: "click" + EVENT_KEY$4 + DATA_API_KEY$4, + KEYDOWN_DATA_API: "keydown" + EVENT_KEY$4 + DATA_API_KEY$4, + KEYUP_DATA_API: "keyup" + EVENT_KEY$4 + DATA_API_KEY$4 + }; + var ClassName$4 = { + DISABLED: 'disabled', + SHOW: 'show', + DROPUP: 'dropup', + DROPRIGHT: 'dropright', + DROPLEFT: 'dropleft', + MENURIGHT: 'dropdown-menu-right', + MENULEFT: 'dropdown-menu-left', + POSITION_STATIC: 'position-static' + }; + var Selector$4 = { + DATA_TOGGLE: '[data-toggle="dropdown"]', + FORM_CHILD: '.dropdown form', + MENU: '.dropdown-menu', + NAVBAR_NAV: '.navbar-nav', + VISIBLE_ITEMS: '.dropdown-menu .dropdown-item:not(.disabled):not(:disabled)' + }; + var AttachmentMap = { + TOP: 'top-start', + TOPEND: 'top-end', + BOTTOM: 'bottom-start', + BOTTOMEND: 'bottom-end', + RIGHT: 'right-start', + RIGHTEND: 'right-end', + LEFT: 'left-start', + LEFTEND: 'left-end' + }; + var Default$2 = { + offset: 0, + flip: true, + boundary: 'scrollParent', + reference: 'toggle', + display: 'dynamic', + popperConfig: null + }; + var DefaultType$2 = { + offset: '(number|string|function)', + flip: 'boolean', + boundary: '(string|element)', + reference: '(string|element)', + display: 'string', + popperConfig: '(null|object)' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Dropdown = + /*#__PURE__*/ + function () { + function Dropdown(element, config) { + this._element = element; + this._popper = null; + this._config = this._getConfig(config); + this._menu = this._getMenuElement(); + this._inNavbar = this._detectNavbar(); + + this._addEventListeners(); + } // Getters + + + var _proto = Dropdown.prototype; + + // Public + _proto.toggle = function toggle() { + if (this._element.disabled || $(this._element).hasClass(ClassName$4.DISABLED)) { + return; + } + + var isActive = $(this._menu).hasClass(ClassName$4.SHOW); + + Dropdown._clearMenus(); + + if (isActive) { + return; + } + + this.show(true); + }; + + _proto.show = function show(usePopper) { + if (usePopper === void 0) { + usePopper = false; + } + + if (this._element.disabled || $(this._element).hasClass(ClassName$4.DISABLED) || $(this._menu).hasClass(ClassName$4.SHOW)) { + return; + } + + var relatedTarget = { + relatedTarget: this._element + }; + var showEvent = $.Event(Event$4.SHOW, relatedTarget); + + var parent = Dropdown._getParentFromElement(this._element); + + $(parent).trigger(showEvent); + + if (showEvent.isDefaultPrevented()) { + return; + } // Disable totally Popper.js for Dropdown in Navbar + + + if (!this._inNavbar && usePopper) { + /** + * Check for Popper dependency + * Popper - https://popper.js.org + */ + if (typeof Popper === 'undefined') { + throw new TypeError('Bootstrap\'s dropdowns require Popper.js (https://popper.js.org/)'); + } + + var referenceElement = this._element; + + if (this._config.reference === 'parent') { + referenceElement = parent; + } else if (Util.isElement(this._config.reference)) { + referenceElement = this._config.reference; // Check if it's jQuery element + + if (typeof this._config.reference.jquery !== 'undefined') { + referenceElement = this._config.reference[0]; + } + } // If boundary is not `scrollParent`, then set position to `static` + // to allow the menu to "escape" the scroll parent's boundaries + // https://github.com/twbs/bootstrap/issues/24251 + + + if (this._config.boundary !== 'scrollParent') { + $(parent).addClass(ClassName$4.POSITION_STATIC); + } + + this._popper = new Popper(referenceElement, this._menu, this._getPopperConfig()); + } // If this is a touch-enabled device we add extra + // empty mouseover listeners to the body's immediate children; + // only needed because of broken event delegation on iOS + // https://www.quirksmode.org/blog/archives/2014/02/mouse_event_bub.html + + + if ('ontouchstart' in document.documentElement && $(parent).closest(Selector$4.NAVBAR_NAV).length === 0) { + $(document.body).children().on('mouseover', null, $.noop); + } + + this._element.focus(); + + this._element.setAttribute('aria-expanded', true); + + $(this._menu).toggleClass(ClassName$4.SHOW); + $(parent).toggleClass(ClassName$4.SHOW).trigger($.Event(Event$4.SHOWN, relatedTarget)); + }; + + _proto.hide = function hide() { + if (this._element.disabled || $(this._element).hasClass(ClassName$4.DISABLED) || !$(this._menu).hasClass(ClassName$4.SHOW)) { + return; + } + + var relatedTarget = { + relatedTarget: this._element + }; + var hideEvent = $.Event(Event$4.HIDE, relatedTarget); + + var parent = Dropdown._getParentFromElement(this._element); + + $(parent).trigger(hideEvent); + + if (hideEvent.isDefaultPrevented()) { + return; + } + + if (this._popper) { + this._popper.destroy(); + } + + $(this._menu).toggleClass(ClassName$4.SHOW); + $(parent).toggleClass(ClassName$4.SHOW).trigger($.Event(Event$4.HIDDEN, relatedTarget)); + }; + + _proto.dispose = function dispose() { + $.removeData(this._element, DATA_KEY$4); + $(this._element).off(EVENT_KEY$4); + this._element = null; + this._menu = null; + + if (this._popper !== null) { + this._popper.destroy(); + + this._popper = null; + } + }; + + _proto.update = function update() { + this._inNavbar = this._detectNavbar(); + + if (this._popper !== null) { + this._popper.scheduleUpdate(); + } + } // Private + ; + + _proto._addEventListeners = function _addEventListeners() { + var _this = this; + + $(this._element).on(Event$4.CLICK, function (event) { + event.preventDefault(); + event.stopPropagation(); + + _this.toggle(); + }); + }; + + _proto._getConfig = function _getConfig(config) { + config = _objectSpread2({}, this.constructor.Default, {}, $(this._element).data(), {}, config); + Util.typeCheckConfig(NAME$4, config, this.constructor.DefaultType); + return config; + }; + + _proto._getMenuElement = function _getMenuElement() { + if (!this._menu) { + var parent = Dropdown._getParentFromElement(this._element); + + if (parent) { + this._menu = parent.querySelector(Selector$4.MENU); + } + } + + return this._menu; + }; + + _proto._getPlacement = function _getPlacement() { + var $parentDropdown = $(this._element.parentNode); + var placement = AttachmentMap.BOTTOM; // Handle dropup + + if ($parentDropdown.hasClass(ClassName$4.DROPUP)) { + placement = AttachmentMap.TOP; + + if ($(this._menu).hasClass(ClassName$4.MENURIGHT)) { + placement = AttachmentMap.TOPEND; + } + } else if ($parentDropdown.hasClass(ClassName$4.DROPRIGHT)) { + placement = AttachmentMap.RIGHT; + } else if ($parentDropdown.hasClass(ClassName$4.DROPLEFT)) { + placement = AttachmentMap.LEFT; + } else if ($(this._menu).hasClass(ClassName$4.MENURIGHT)) { + placement = AttachmentMap.BOTTOMEND; + } + + return placement; + }; + + _proto._detectNavbar = function _detectNavbar() { + return $(this._element).closest('.navbar').length > 0; + }; + + _proto._getOffset = function _getOffset() { + var _this2 = this; + + var offset = {}; + + if (typeof this._config.offset === 'function') { + offset.fn = function (data) { + data.offsets = _objectSpread2({}, data.offsets, {}, _this2._config.offset(data.offsets, _this2._element) || {}); + return data; + }; + } else { + offset.offset = this._config.offset; + } + + return offset; + }; + + _proto._getPopperConfig = function _getPopperConfig() { + var popperConfig = { + placement: this._getPlacement(), + modifiers: { + offset: this._getOffset(), + flip: { + enabled: this._config.flip + }, + preventOverflow: { + boundariesElement: this._config.boundary + } + } + }; // Disable Popper.js if we have a static display + + if (this._config.display === 'static') { + popperConfig.modifiers.applyStyle = { + enabled: false + }; + } + + return _objectSpread2({}, popperConfig, {}, this._config.popperConfig); + } // Static + ; + + Dropdown._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var data = $(this).data(DATA_KEY$4); + + var _config = typeof config === 'object' ? config : null; + + if (!data) { + data = new Dropdown(this, _config); + $(this).data(DATA_KEY$4, data); + } + + if (typeof config === 'string') { + if (typeof data[config] === 'undefined') { + throw new TypeError("No method named \"" + config + "\""); + } + + data[config](); + } + }); + }; + + Dropdown._clearMenus = function _clearMenus(event) { + if (event && (event.which === RIGHT_MOUSE_BUTTON_WHICH || event.type === 'keyup' && event.which !== TAB_KEYCODE)) { + return; + } + + var toggles = [].slice.call(document.querySelectorAll(Selector$4.DATA_TOGGLE)); + + for (var i = 0, len = toggles.length; i < len; i++) { + var parent = Dropdown._getParentFromElement(toggles[i]); + + var context = $(toggles[i]).data(DATA_KEY$4); + var relatedTarget = { + relatedTarget: toggles[i] + }; + + if (event && event.type === 'click') { + relatedTarget.clickEvent = event; + } + + if (!context) { + continue; + } + + var dropdownMenu = context._menu; + + if (!$(parent).hasClass(ClassName$4.SHOW)) { + continue; + } + + if (event && (event.type === 'click' && /input|textarea/i.test(event.target.tagName) || event.type === 'keyup' && event.which === TAB_KEYCODE) && $.contains(parent, event.target)) { + continue; + } + + var hideEvent = $.Event(Event$4.HIDE, relatedTarget); + $(parent).trigger(hideEvent); + + if (hideEvent.isDefaultPrevented()) { + continue; + } // If this is a touch-enabled device we remove the extra + // empty mouseover listeners we added for iOS support + + + if ('ontouchstart' in document.documentElement) { + $(document.body).children().off('mouseover', null, $.noop); + } + + toggles[i].setAttribute('aria-expanded', 'false'); + + if (context._popper) { + context._popper.destroy(); + } + + $(dropdownMenu).removeClass(ClassName$4.SHOW); + $(parent).removeClass(ClassName$4.SHOW).trigger($.Event(Event$4.HIDDEN, relatedTarget)); + } + }; + + Dropdown._getParentFromElement = function _getParentFromElement(element) { + var parent; + var selector = Util.getSelectorFromElement(element); + + if (selector) { + parent = document.querySelector(selector); + } + + return parent || element.parentNode; + } // eslint-disable-next-line complexity + ; + + Dropdown._dataApiKeydownHandler = function _dataApiKeydownHandler(event) { + // If not input/textarea: + // - And not a key in REGEXP_KEYDOWN => not a dropdown command + // If input/textarea: + // - If space key => not a dropdown command + // - If key is other than escape + // - If key is not up or down => not a dropdown command + // - If trigger inside the menu => not a dropdown command + if (/input|textarea/i.test(event.target.tagName) ? event.which === SPACE_KEYCODE || event.which !== ESCAPE_KEYCODE && (event.which !== ARROW_DOWN_KEYCODE && event.which !== ARROW_UP_KEYCODE || $(event.target).closest(Selector$4.MENU).length) : !REGEXP_KEYDOWN.test(event.which)) { + return; + } + + event.preventDefault(); + event.stopPropagation(); + + if (this.disabled || $(this).hasClass(ClassName$4.DISABLED)) { + return; + } + + var parent = Dropdown._getParentFromElement(this); + + var isActive = $(parent).hasClass(ClassName$4.SHOW); + + if (!isActive && event.which === ESCAPE_KEYCODE) { + return; + } + + if (!isActive || isActive && (event.which === ESCAPE_KEYCODE || event.which === SPACE_KEYCODE)) { + if (event.which === ESCAPE_KEYCODE) { + var toggle = parent.querySelector(Selector$4.DATA_TOGGLE); + $(toggle).trigger('focus'); + } + + $(this).trigger('click'); + return; + } + + var items = [].slice.call(parent.querySelectorAll(Selector$4.VISIBLE_ITEMS)).filter(function (item) { + return $(item).is(':visible'); + }); + + if (items.length === 0) { + return; + } + + var index = items.indexOf(event.target); + + if (event.which === ARROW_UP_KEYCODE && index > 0) { + // Up + index--; + } + + if (event.which === ARROW_DOWN_KEYCODE && index < items.length - 1) { + // Down + index++; + } + + if (index < 0) { + index = 0; + } + + items[index].focus(); + }; + + _createClass(Dropdown, null, [{ + key: "VERSION", + get: function get() { + return VERSION$4; + } + }, { + key: "Default", + get: function get() { + return Default$2; + } + }, { + key: "DefaultType", + get: function get() { + return DefaultType$2; + } + }]); + + return Dropdown; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event$4.KEYDOWN_DATA_API, Selector$4.DATA_TOGGLE, Dropdown._dataApiKeydownHandler).on(Event$4.KEYDOWN_DATA_API, Selector$4.MENU, Dropdown._dataApiKeydownHandler).on(Event$4.CLICK_DATA_API + " " + Event$4.KEYUP_DATA_API, Dropdown._clearMenus).on(Event$4.CLICK_DATA_API, Selector$4.DATA_TOGGLE, function (event) { + event.preventDefault(); + event.stopPropagation(); + + Dropdown._jQueryInterface.call($(this), 'toggle'); + }).on(Event$4.CLICK_DATA_API, Selector$4.FORM_CHILD, function (e) { + e.stopPropagation(); + }); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME$4] = Dropdown._jQueryInterface; + $.fn[NAME$4].Constructor = Dropdown; + + $.fn[NAME$4].noConflict = function () { + $.fn[NAME$4] = JQUERY_NO_CONFLICT$4; + return Dropdown._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$5 = 'modal'; + var VERSION$5 = '4.4.1'; + var DATA_KEY$5 = 'bs.modal'; + var EVENT_KEY$5 = "." + DATA_KEY$5; + var DATA_API_KEY$5 = '.data-api'; + var JQUERY_NO_CONFLICT$5 = $.fn[NAME$5]; + var ESCAPE_KEYCODE$1 = 27; // KeyboardEvent.which value for Escape (Esc) key + + var Default$3 = { + backdrop: true, + keyboard: true, + focus: true, + show: true + }; + var DefaultType$3 = { + backdrop: '(boolean|string)', + keyboard: 'boolean', + focus: 'boolean', + show: 'boolean' + }; + var Event$5 = { + HIDE: "hide" + EVENT_KEY$5, + HIDE_PREVENTED: "hidePrevented" + EVENT_KEY$5, + HIDDEN: "hidden" + EVENT_KEY$5, + SHOW: "show" + EVENT_KEY$5, + SHOWN: "shown" + EVENT_KEY$5, + FOCUSIN: "focusin" + EVENT_KEY$5, + RESIZE: "resize" + EVENT_KEY$5, + CLICK_DISMISS: "click.dismiss" + EVENT_KEY$5, + KEYDOWN_DISMISS: "keydown.dismiss" + EVENT_KEY$5, + MOUSEUP_DISMISS: "mouseup.dismiss" + EVENT_KEY$5, + MOUSEDOWN_DISMISS: "mousedown.dismiss" + EVENT_KEY$5, + CLICK_DATA_API: "click" + EVENT_KEY$5 + DATA_API_KEY$5 + }; + var ClassName$5 = { + SCROLLABLE: 'modal-dialog-scrollable', + SCROLLBAR_MEASURER: 'modal-scrollbar-measure', + BACKDROP: 'modal-backdrop', + OPEN: 'modal-open', + FADE: 'fade', + SHOW: 'show', + STATIC: 'modal-static' + }; + var Selector$5 = { + DIALOG: '.modal-dialog', + MODAL_BODY: '.modal-body', + DATA_TOGGLE: '[data-toggle="modal"]', + DATA_DISMISS: '[data-dismiss="modal"]', + FIXED_CONTENT: '.fixed-top, .fixed-bottom, .is-fixed, .sticky-top', + STICKY_CONTENT: '.sticky-top' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Modal = + /*#__PURE__*/ + function () { + function Modal(element, config) { + this._config = this._getConfig(config); + this._element = element; + this._dialog = element.querySelector(Selector$5.DIALOG); + this._backdrop = null; + this._isShown = false; + this._isBodyOverflowing = false; + this._ignoreBackdropClick = false; + this._isTransitioning = false; + this._scrollbarWidth = 0; + } // Getters + + + var _proto = Modal.prototype; + + // Public + _proto.toggle = function toggle(relatedTarget) { + return this._isShown ? this.hide() : this.show(relatedTarget); + }; + + _proto.show = function show(relatedTarget) { + var _this = this; + + if (this._isShown || this._isTransitioning) { + return; + } + + if ($(this._element).hasClass(ClassName$5.FADE)) { + this._isTransitioning = true; + } + + var showEvent = $.Event(Event$5.SHOW, { + relatedTarget: relatedTarget + }); + $(this._element).trigger(showEvent); + + if (this._isShown || showEvent.isDefaultPrevented()) { + return; + } + + this._isShown = true; + + this._checkScrollbar(); + + this._setScrollbar(); + + this._adjustDialog(); + + this._setEscapeEvent(); + + this._setResizeEvent(); + + $(this._element).on(Event$5.CLICK_DISMISS, Selector$5.DATA_DISMISS, function (event) { + return _this.hide(event); + }); + $(this._dialog).on(Event$5.MOUSEDOWN_DISMISS, function () { + $(_this._element).one(Event$5.MOUSEUP_DISMISS, function (event) { + if ($(event.target).is(_this._element)) { + _this._ignoreBackdropClick = true; + } + }); + }); + + this._showBackdrop(function () { + return _this._showElement(relatedTarget); + }); + }; + + _proto.hide = function hide(event) { + var _this2 = this; + + if (event) { + event.preventDefault(); + } + + if (!this._isShown || this._isTransitioning) { + return; + } + + var hideEvent = $.Event(Event$5.HIDE); + $(this._element).trigger(hideEvent); + + if (!this._isShown || hideEvent.isDefaultPrevented()) { + return; + } + + this._isShown = false; + var transition = $(this._element).hasClass(ClassName$5.FADE); + + if (transition) { + this._isTransitioning = true; + } + + this._setEscapeEvent(); + + this._setResizeEvent(); + + $(document).off(Event$5.FOCUSIN); + $(this._element).removeClass(ClassName$5.SHOW); + $(this._element).off(Event$5.CLICK_DISMISS); + $(this._dialog).off(Event$5.MOUSEDOWN_DISMISS); + + if (transition) { + var transitionDuration = Util.getTransitionDurationFromElement(this._element); + $(this._element).one(Util.TRANSITION_END, function (event) { + return _this2._hideModal(event); + }).emulateTransitionEnd(transitionDuration); + } else { + this._hideModal(); + } + }; + + _proto.dispose = function dispose() { + [window, this._element, this._dialog].forEach(function (htmlElement) { + return $(htmlElement).off(EVENT_KEY$5); + }); + /** + * `document` has 2 events `Event.FOCUSIN` and `Event.CLICK_DATA_API` + * Do not move `document` in `htmlElements` array + * It will remove `Event.CLICK_DATA_API` event that should remain + */ + + $(document).off(Event$5.FOCUSIN); + $.removeData(this._element, DATA_KEY$5); + this._config = null; + this._element = null; + this._dialog = null; + this._backdrop = null; + this._isShown = null; + this._isBodyOverflowing = null; + this._ignoreBackdropClick = null; + this._isTransitioning = null; + this._scrollbarWidth = null; + }; + + _proto.handleUpdate = function handleUpdate() { + this._adjustDialog(); + } // Private + ; + + _proto._getConfig = function _getConfig(config) { + config = _objectSpread2({}, Default$3, {}, config); + Util.typeCheckConfig(NAME$5, config, DefaultType$3); + return config; + }; + + _proto._triggerBackdropTransition = function _triggerBackdropTransition() { + var _this3 = this; + + if (this._config.backdrop === 'static') { + var hideEventPrevented = $.Event(Event$5.HIDE_PREVENTED); + $(this._element).trigger(hideEventPrevented); + + if (hideEventPrevented.defaultPrevented) { + return; + } + + this._element.classList.add(ClassName$5.STATIC); + + var modalTransitionDuration = Util.getTransitionDurationFromElement(this._element); + $(this._element).one(Util.TRANSITION_END, function () { + _this3._element.classList.remove(ClassName$5.STATIC); + }).emulateTransitionEnd(modalTransitionDuration); + + this._element.focus(); + } else { + this.hide(); + } + }; + + _proto._showElement = function _showElement(relatedTarget) { + var _this4 = this; + + var transition = $(this._element).hasClass(ClassName$5.FADE); + var modalBody = this._dialog ? this._dialog.querySelector(Selector$5.MODAL_BODY) : null; + + if (!this._element.parentNode || this._element.parentNode.nodeType !== Node.ELEMENT_NODE) { + // Don't move modal's DOM position + document.body.appendChild(this._element); + } + + this._element.style.display = 'block'; + + this._element.removeAttribute('aria-hidden'); + + this._element.setAttribute('aria-modal', true); + + if ($(this._dialog).hasClass(ClassName$5.SCROLLABLE) && modalBody) { + modalBody.scrollTop = 0; + } else { + this._element.scrollTop = 0; + } + + if (transition) { + Util.reflow(this._element); + } + + $(this._element).addClass(ClassName$5.SHOW); + + if (this._config.focus) { + this._enforceFocus(); + } + + var shownEvent = $.Event(Event$5.SHOWN, { + relatedTarget: relatedTarget + }); + + var transitionComplete = function transitionComplete() { + if (_this4._config.focus) { + _this4._element.focus(); + } + + _this4._isTransitioning = false; + $(_this4._element).trigger(shownEvent); + }; + + if (transition) { + var transitionDuration = Util.getTransitionDurationFromElement(this._dialog); + $(this._dialog).one(Util.TRANSITION_END, transitionComplete).emulateTransitionEnd(transitionDuration); + } else { + transitionComplete(); + } + }; + + _proto._enforceFocus = function _enforceFocus() { + var _this5 = this; + + $(document).off(Event$5.FOCUSIN) // Guard against infinite focus loop + .on(Event$5.FOCUSIN, function (event) { + if (document !== event.target && _this5._element !== event.target && $(_this5._element).has(event.target).length === 0) { + _this5._element.focus(); + } + }); + }; + + _proto._setEscapeEvent = function _setEscapeEvent() { + var _this6 = this; + + if (this._isShown && this._config.keyboard) { + $(this._element).on(Event$5.KEYDOWN_DISMISS, function (event) { + if (event.which === ESCAPE_KEYCODE$1) { + _this6._triggerBackdropTransition(); + } + }); + } else if (!this._isShown) { + $(this._element).off(Event$5.KEYDOWN_DISMISS); + } + }; + + _proto._setResizeEvent = function _setResizeEvent() { + var _this7 = this; + + if (this._isShown) { + $(window).on(Event$5.RESIZE, function (event) { + return _this7.handleUpdate(event); + }); + } else { + $(window).off(Event$5.RESIZE); + } + }; + + _proto._hideModal = function _hideModal() { + var _this8 = this; + + this._element.style.display = 'none'; + + this._element.setAttribute('aria-hidden', true); + + this._element.removeAttribute('aria-modal'); + + this._isTransitioning = false; + + this._showBackdrop(function () { + $(document.body).removeClass(ClassName$5.OPEN); + + _this8._resetAdjustments(); + + _this8._resetScrollbar(); + + $(_this8._element).trigger(Event$5.HIDDEN); + }); + }; + + _proto._removeBackdrop = function _removeBackdrop() { + if (this._backdrop) { + $(this._backdrop).remove(); + this._backdrop = null; + } + }; + + _proto._showBackdrop = function _showBackdrop(callback) { + var _this9 = this; + + var animate = $(this._element).hasClass(ClassName$5.FADE) ? ClassName$5.FADE : ''; + + if (this._isShown && this._config.backdrop) { + this._backdrop = document.createElement('div'); + this._backdrop.className = ClassName$5.BACKDROP; + + if (animate) { + this._backdrop.classList.add(animate); + } + + $(this._backdrop).appendTo(document.body); + $(this._element).on(Event$5.CLICK_DISMISS, function (event) { + if (_this9._ignoreBackdropClick) { + _this9._ignoreBackdropClick = false; + return; + } + + if (event.target !== event.currentTarget) { + return; + } + + _this9._triggerBackdropTransition(); + }); + + if (animate) { + Util.reflow(this._backdrop); + } + + $(this._backdrop).addClass(ClassName$5.SHOW); + + if (!callback) { + return; + } + + if (!animate) { + callback(); + return; + } + + var backdropTransitionDuration = Util.getTransitionDurationFromElement(this._backdrop); + $(this._backdrop).one(Util.TRANSITION_END, callback).emulateTransitionEnd(backdropTransitionDuration); + } else if (!this._isShown && this._backdrop) { + $(this._backdrop).removeClass(ClassName$5.SHOW); + + var callbackRemove = function callbackRemove() { + _this9._removeBackdrop(); + + if (callback) { + callback(); + } + }; + + if ($(this._element).hasClass(ClassName$5.FADE)) { + var _backdropTransitionDuration = Util.getTransitionDurationFromElement(this._backdrop); + + $(this._backdrop).one(Util.TRANSITION_END, callbackRemove).emulateTransitionEnd(_backdropTransitionDuration); + } else { + callbackRemove(); + } + } else if (callback) { + callback(); + } + } // ---------------------------------------------------------------------- + // the following methods are used to handle overflowing modals + // todo (fat): these should probably be refactored out of modal.js + // ---------------------------------------------------------------------- + ; + + _proto._adjustDialog = function _adjustDialog() { + var isModalOverflowing = this._element.scrollHeight > document.documentElement.clientHeight; + + if (!this._isBodyOverflowing && isModalOverflowing) { + this._element.style.paddingLeft = this._scrollbarWidth + "px"; + } + + if (this._isBodyOverflowing && !isModalOverflowing) { + this._element.style.paddingRight = this._scrollbarWidth + "px"; + } + }; + + _proto._resetAdjustments = function _resetAdjustments() { + this._element.style.paddingLeft = ''; + this._element.style.paddingRight = ''; + }; + + _proto._checkScrollbar = function _checkScrollbar() { + var rect = document.body.getBoundingClientRect(); + this._isBodyOverflowing = rect.left + rect.right < window.innerWidth; + this._scrollbarWidth = this._getScrollbarWidth(); + }; + + _proto._setScrollbar = function _setScrollbar() { + var _this10 = this; + + if (this._isBodyOverflowing) { + // Note: DOMNode.style.paddingRight returns the actual value or '' if not set + // while $(DOMNode).css('padding-right') returns the calculated value or 0 if not set + var fixedContent = [].slice.call(document.querySelectorAll(Selector$5.FIXED_CONTENT)); + var stickyContent = [].slice.call(document.querySelectorAll(Selector$5.STICKY_CONTENT)); // Adjust fixed content padding + + $(fixedContent).each(function (index, element) { + var actualPadding = element.style.paddingRight; + var calculatedPadding = $(element).css('padding-right'); + $(element).data('padding-right', actualPadding).css('padding-right', parseFloat(calculatedPadding) + _this10._scrollbarWidth + "px"); + }); // Adjust sticky content margin + + $(stickyContent).each(function (index, element) { + var actualMargin = element.style.marginRight; + var calculatedMargin = $(element).css('margin-right'); + $(element).data('margin-right', actualMargin).css('margin-right', parseFloat(calculatedMargin) - _this10._scrollbarWidth + "px"); + }); // Adjust body padding + + var actualPadding = document.body.style.paddingRight; + var calculatedPadding = $(document.body).css('padding-right'); + $(document.body).data('padding-right', actualPadding).css('padding-right', parseFloat(calculatedPadding) + this._scrollbarWidth + "px"); + } + + $(document.body).addClass(ClassName$5.OPEN); + }; + + _proto._resetScrollbar = function _resetScrollbar() { + // Restore fixed content padding + var fixedContent = [].slice.call(document.querySelectorAll(Selector$5.FIXED_CONTENT)); + $(fixedContent).each(function (index, element) { + var padding = $(element).data('padding-right'); + $(element).removeData('padding-right'); + element.style.paddingRight = padding ? padding : ''; + }); // Restore sticky content + + var elements = [].slice.call(document.querySelectorAll("" + Selector$5.STICKY_CONTENT)); + $(elements).each(function (index, element) { + var margin = $(element).data('margin-right'); + + if (typeof margin !== 'undefined') { + $(element).css('margin-right', margin).removeData('margin-right'); + } + }); // Restore body padding + + var padding = $(document.body).data('padding-right'); + $(document.body).removeData('padding-right'); + document.body.style.paddingRight = padding ? padding : ''; + }; + + _proto._getScrollbarWidth = function _getScrollbarWidth() { + // thx d.walsh + var scrollDiv = document.createElement('div'); + scrollDiv.className = ClassName$5.SCROLLBAR_MEASURER; + document.body.appendChild(scrollDiv); + var scrollbarWidth = scrollDiv.getBoundingClientRect().width - scrollDiv.clientWidth; + document.body.removeChild(scrollDiv); + return scrollbarWidth; + } // Static + ; + + Modal._jQueryInterface = function _jQueryInterface(config, relatedTarget) { + return this.each(function () { + var data = $(this).data(DATA_KEY$5); + + var _config = _objectSpread2({}, Default$3, {}, $(this).data(), {}, typeof config === 'object' && config ? config : {}); + + if (!data) { + data = new Modal(this, _config); + $(this).data(DATA_KEY$5, data); + } + + if (typeof config === 'string') { + if (typeof data[config] === 'undefined') { + throw new TypeError("No method named \"" + config + "\""); + } + + data[config](relatedTarget); + } else if (_config.show) { + data.show(relatedTarget); + } + }); + }; + + _createClass(Modal, null, [{ + key: "VERSION", + get: function get() { + return VERSION$5; + } + }, { + key: "Default", + get: function get() { + return Default$3; + } + }]); + + return Modal; + }(); + /** + * ------------------------------------------------------------------------ + * Data Api implementation + * ------------------------------------------------------------------------ + */ + + + $(document).on(Event$5.CLICK_DATA_API, Selector$5.DATA_TOGGLE, function (event) { + var _this11 = this; + + var target; + var selector = Util.getSelectorFromElement(this); + + if (selector) { + target = document.querySelector(selector); + } + + var config = $(target).data(DATA_KEY$5) ? 'toggle' : _objectSpread2({}, $(target).data(), {}, $(this).data()); + + if (this.tagName === 'A' || this.tagName === 'AREA') { + event.preventDefault(); + } + + var $target = $(target).one(Event$5.SHOW, function (showEvent) { + if (showEvent.isDefaultPrevented()) { + // Only register focus restorer if modal will actually get shown + return; + } + + $target.one(Event$5.HIDDEN, function () { + if ($(_this11).is(':visible')) { + _this11.focus(); + } + }); + }); + + Modal._jQueryInterface.call($(target), config, this); + }); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + $.fn[NAME$5] = Modal._jQueryInterface; + $.fn[NAME$5].Constructor = Modal; + + $.fn[NAME$5].noConflict = function () { + $.fn[NAME$5] = JQUERY_NO_CONFLICT$5; + return Modal._jQueryInterface; + }; + + /** + * -------------------------------------------------------------------------- + * Bootstrap (v4.4.1): tools/sanitizer.js + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + * -------------------------------------------------------------------------- + */ + var uriAttrs = ['background', 'cite', 'href', 'itemtype', 'longdesc', 'poster', 'src', 'xlink:href']; + var ARIA_ATTRIBUTE_PATTERN = /^aria-[\w-]*$/i; + var DefaultWhitelist = { + // Global attributes allowed on any supplied element below. + '*': ['class', 'dir', 'id', 'lang', 'role', ARIA_ATTRIBUTE_PATTERN], + a: ['target', 'href', 'title', 'rel'], + area: [], + b: [], + br: [], + col: [], + code: [], + div: [], + em: [], + hr: [], + h1: [], + h2: [], + h3: [], + h4: [], + h5: [], + h6: [], + i: [], + img: ['src', 'alt', 'title', 'width', 'height'], + li: [], + ol: [], + p: [], + pre: [], + s: [], + small: [], + span: [], + sub: [], + sup: [], + strong: [], + u: [], + ul: [] + }; + /** + * A pattern that recognizes a commonly useful subset of URLs that are safe. + * + * Shoutout to Angular 7 https://github.com/angular/angular/blob/7.2.4/packages/core/src/sanitization/url_sanitizer.ts + */ + + var SAFE_URL_PATTERN = /^(?:(?:https?|mailto|ftp|tel|file):|[^&:/?#]*(?:[/?#]|$))/gi; + /** + * A pattern that matches safe data URLs. Only matches image, video and audio types. + * + * Shoutout to Angular 7 https://github.com/angular/angular/blob/7.2.4/packages/core/src/sanitization/url_sanitizer.ts + */ + + var DATA_URL_PATTERN = /^data:(?:image\/(?:bmp|gif|jpeg|jpg|png|tiff|webp)|video\/(?:mpeg|mp4|ogg|webm)|audio\/(?:mp3|oga|ogg|opus));base64,[a-z0-9+/]+=*$/i; + + function allowedAttribute(attr, allowedAttributeList) { + var attrName = attr.nodeName.toLowerCase(); + + if (allowedAttributeList.indexOf(attrName) !== -1) { + if (uriAttrs.indexOf(attrName) !== -1) { + return Boolean(attr.nodeValue.match(SAFE_URL_PATTERN) || attr.nodeValue.match(DATA_URL_PATTERN)); + } + + return true; + } + + var regExp = allowedAttributeList.filter(function (attrRegex) { + return attrRegex instanceof RegExp; + }); // Check if a regular expression validates the attribute. + + for (var i = 0, l = regExp.length; i < l; i++) { + if (attrName.match(regExp[i])) { + return true; + } + } + + return false; + } + + function sanitizeHtml(unsafeHtml, whiteList, sanitizeFn) { + if (unsafeHtml.length === 0) { + return unsafeHtml; + } + + if (sanitizeFn && typeof sanitizeFn === 'function') { + return sanitizeFn(unsafeHtml); + } + + var domParser = new window.DOMParser(); + var createdDocument = domParser.parseFromString(unsafeHtml, 'text/html'); + var whitelistKeys = Object.keys(whiteList); + var elements = [].slice.call(createdDocument.body.querySelectorAll('*')); + + var _loop = function _loop(i, len) { + var el = elements[i]; + var elName = el.nodeName.toLowerCase(); + + if (whitelistKeys.indexOf(el.nodeName.toLowerCase()) === -1) { + el.parentNode.removeChild(el); + return "continue"; + } + + var attributeList = [].slice.call(el.attributes); + var whitelistedAttributes = [].concat(whiteList['*'] || [], whiteList[elName] || []); + attributeList.forEach(function (attr) { + if (!allowedAttribute(attr, whitelistedAttributes)) { + el.removeAttribute(attr.nodeName); + } + }); + }; + + for (var i = 0, len = elements.length; i < len; i++) { + var _ret = _loop(i); + + if (_ret === "continue") continue; + } + + return createdDocument.body.innerHTML; + } + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$6 = 'tooltip'; + var VERSION$6 = '4.4.1'; + var DATA_KEY$6 = 'bs.tooltip'; + var EVENT_KEY$6 = "." + DATA_KEY$6; + var JQUERY_NO_CONFLICT$6 = $.fn[NAME$6]; + var CLASS_PREFIX = 'bs-tooltip'; + var BSCLS_PREFIX_REGEX = new RegExp("(^|\\s)" + CLASS_PREFIX + "\\S+", 'g'); + var DISALLOWED_ATTRIBUTES = ['sanitize', 'whiteList', 'sanitizeFn']; + var DefaultType$4 = { + animation: 'boolean', + template: 'string', + title: '(string|element|function)', + trigger: 'string', + delay: '(number|object)', + html: 'boolean', + selector: '(string|boolean)', + placement: '(string|function)', + offset: '(number|string|function)', + container: '(string|element|boolean)', + fallbackPlacement: '(string|array)', + boundary: '(string|element)', + sanitize: 'boolean', + sanitizeFn: '(null|function)', + whiteList: 'object', + popperConfig: '(null|object)' + }; + var AttachmentMap$1 = { + AUTO: 'auto', + TOP: 'top', + RIGHT: 'right', + BOTTOM: 'bottom', + LEFT: 'left' + }; + var Default$4 = { + animation: true, + template: '', + trigger: 'hover focus', + title: '', + delay: 0, + html: false, + selector: false, + placement: 'top', + offset: 0, + container: false, + fallbackPlacement: 'flip', + boundary: 'scrollParent', + sanitize: true, + sanitizeFn: null, + whiteList: DefaultWhitelist, + popperConfig: null + }; + var HoverState = { + SHOW: 'show', + OUT: 'out' + }; + var Event$6 = { + HIDE: "hide" + EVENT_KEY$6, + HIDDEN: "hidden" + EVENT_KEY$6, + SHOW: "show" + EVENT_KEY$6, + SHOWN: "shown" + EVENT_KEY$6, + INSERTED: "inserted" + EVENT_KEY$6, + CLICK: "click" + EVENT_KEY$6, + FOCUSIN: "focusin" + EVENT_KEY$6, + FOCUSOUT: "focusout" + EVENT_KEY$6, + MOUSEENTER: "mouseenter" + EVENT_KEY$6, + MOUSELEAVE: "mouseleave" + EVENT_KEY$6 + }; + var ClassName$6 = { + FADE: 'fade', + SHOW: 'show' + }; + var Selector$6 = { + TOOLTIP: '.tooltip', + TOOLTIP_INNER: '.tooltip-inner', + ARROW: '.arrow' + }; + var Trigger = { + HOVER: 'hover', + FOCUS: 'focus', + CLICK: 'click', + MANUAL: 'manual' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Tooltip = + /*#__PURE__*/ + function () { + function Tooltip(element, config) { + if (typeof Popper === 'undefined') { + throw new TypeError('Bootstrap\'s tooltips require Popper.js (https://popper.js.org/)'); + } // private + + + this._isEnabled = true; + this._timeout = 0; + this._hoverState = ''; + this._activeTrigger = {}; + this._popper = null; // Protected + + this.element = element; + this.config = this._getConfig(config); + this.tip = null; + + this._setListeners(); + } // Getters + + + var _proto = Tooltip.prototype; + + // Public + _proto.enable = function enable() { + this._isEnabled = true; + }; + + _proto.disable = function disable() { + this._isEnabled = false; + }; + + _proto.toggleEnabled = function toggleEnabled() { + this._isEnabled = !this._isEnabled; + }; + + _proto.toggle = function toggle(event) { + if (!this._isEnabled) { + return; + } + + if (event) { + var dataKey = this.constructor.DATA_KEY; + var context = $(event.currentTarget).data(dataKey); + + if (!context) { + context = new this.constructor(event.currentTarget, this._getDelegateConfig()); + $(event.currentTarget).data(dataKey, context); + } + + context._activeTrigger.click = !context._activeTrigger.click; + + if (context._isWithActiveTrigger()) { + context._enter(null, context); + } else { + context._leave(null, context); + } + } else { + if ($(this.getTipElement()).hasClass(ClassName$6.SHOW)) { + this._leave(null, this); + + return; + } + + this._enter(null, this); + } + }; + + _proto.dispose = function dispose() { + clearTimeout(this._timeout); + $.removeData(this.element, this.constructor.DATA_KEY); + $(this.element).off(this.constructor.EVENT_KEY); + $(this.element).closest('.modal').off('hide.bs.modal', this._hideModalHandler); + + if (this.tip) { + $(this.tip).remove(); + } + + this._isEnabled = null; + this._timeout = null; + this._hoverState = null; + this._activeTrigger = null; + + if (this._popper) { + this._popper.destroy(); + } + + this._popper = null; + this.element = null; + this.config = null; + this.tip = null; + }; + + _proto.show = function show() { + var _this = this; + + if ($(this.element).css('display') === 'none') { + throw new Error('Please use show on visible elements'); + } + + var showEvent = $.Event(this.constructor.Event.SHOW); + + if (this.isWithContent() && this._isEnabled) { + $(this.element).trigger(showEvent); + var shadowRoot = Util.findShadowRoot(this.element); + var isInTheDom = $.contains(shadowRoot !== null ? shadowRoot : this.element.ownerDocument.documentElement, this.element); + + if (showEvent.isDefaultPrevented() || !isInTheDom) { + return; + } + + var tip = this.getTipElement(); + var tipId = Util.getUID(this.constructor.NAME); + tip.setAttribute('id', tipId); + this.element.setAttribute('aria-describedby', tipId); + this.setContent(); + + if (this.config.animation) { + $(tip).addClass(ClassName$6.FADE); + } + + var placement = typeof this.config.placement === 'function' ? this.config.placement.call(this, tip, this.element) : this.config.placement; + + var attachment = this._getAttachment(placement); + + this.addAttachmentClass(attachment); + + var container = this._getContainer(); + + $(tip).data(this.constructor.DATA_KEY, this); + + if (!$.contains(this.element.ownerDocument.documentElement, this.tip)) { + $(tip).appendTo(container); + } + + $(this.element).trigger(this.constructor.Event.INSERTED); + this._popper = new Popper(this.element, tip, this._getPopperConfig(attachment)); + $(tip).addClass(ClassName$6.SHOW); // If this is a touch-enabled device we add extra + // empty mouseover listeners to the body's immediate children; + // only needed because of broken event delegation on iOS + // https://www.quirksmode.org/blog/archives/2014/02/mouse_event_bub.html + + if ('ontouchstart' in document.documentElement) { + $(document.body).children().on('mouseover', null, $.noop); + } + + var complete = function complete() { + if (_this.config.animation) { + _this._fixTransition(); + } + + var prevHoverState = _this._hoverState; + _this._hoverState = null; + $(_this.element).trigger(_this.constructor.Event.SHOWN); + + if (prevHoverState === HoverState.OUT) { + _this._leave(null, _this); + } + }; + + if ($(this.tip).hasClass(ClassName$6.FADE)) { + var transitionDuration = Util.getTransitionDurationFromElement(this.tip); + $(this.tip).one(Util.TRANSITION_END, complete).emulateTransitionEnd(transitionDuration); + } else { + complete(); + } + } + }; + + _proto.hide = function hide(callback) { + var _this2 = this; + + var tip = this.getTipElement(); + var hideEvent = $.Event(this.constructor.Event.HIDE); + + var complete = function complete() { + if (_this2._hoverState !== HoverState.SHOW && tip.parentNode) { + tip.parentNode.removeChild(tip); + } + + _this2._cleanTipClass(); + + _this2.element.removeAttribute('aria-describedby'); + + $(_this2.element).trigger(_this2.constructor.Event.HIDDEN); + + if (_this2._popper !== null) { + _this2._popper.destroy(); + } + + if (callback) { + callback(); + } + }; + + $(this.element).trigger(hideEvent); + + if (hideEvent.isDefaultPrevented()) { + return; + } + + $(tip).removeClass(ClassName$6.SHOW); // If this is a touch-enabled device we remove the extra + // empty mouseover listeners we added for iOS support + + if ('ontouchstart' in document.documentElement) { + $(document.body).children().off('mouseover', null, $.noop); + } + + this._activeTrigger[Trigger.CLICK] = false; + this._activeTrigger[Trigger.FOCUS] = false; + this._activeTrigger[Trigger.HOVER] = false; + + if ($(this.tip).hasClass(ClassName$6.FADE)) { + var transitionDuration = Util.getTransitionDurationFromElement(tip); + $(tip).one(Util.TRANSITION_END, complete).emulateTransitionEnd(transitionDuration); + } else { + complete(); + } + + this._hoverState = ''; + }; + + _proto.update = function update() { + if (this._popper !== null) { + this._popper.scheduleUpdate(); + } + } // Protected + ; + + _proto.isWithContent = function isWithContent() { + return Boolean(this.getTitle()); + }; + + _proto.addAttachmentClass = function addAttachmentClass(attachment) { + $(this.getTipElement()).addClass(CLASS_PREFIX + "-" + attachment); + }; + + _proto.getTipElement = function getTipElement() { + this.tip = this.tip || $(this.config.template)[0]; + return this.tip; + }; + + _proto.setContent = function setContent() { + var tip = this.getTipElement(); + this.setElementContent($(tip.querySelectorAll(Selector$6.TOOLTIP_INNER)), this.getTitle()); + $(tip).removeClass(ClassName$6.FADE + " " + ClassName$6.SHOW); + }; + + _proto.setElementContent = function setElementContent($element, content) { + if (typeof content === 'object' && (content.nodeType || content.jquery)) { + // Content is a DOM node or a jQuery + if (this.config.html) { + if (!$(content).parent().is($element)) { + $element.empty().append(content); + } + } else { + $element.text($(content).text()); + } + + return; + } + + if (this.config.html) { + if (this.config.sanitize) { + content = sanitizeHtml(content, this.config.whiteList, this.config.sanitizeFn); + } + + $element.html(content); + } else { + $element.text(content); + } + }; + + _proto.getTitle = function getTitle() { + var title = this.element.getAttribute('data-original-title'); + + if (!title) { + title = typeof this.config.title === 'function' ? this.config.title.call(this.element) : this.config.title; + } + + return title; + } // Private + ; + + _proto._getPopperConfig = function _getPopperConfig(attachment) { + var _this3 = this; + + var defaultBsConfig = { + placement: attachment, + modifiers: { + offset: this._getOffset(), + flip: { + behavior: this.config.fallbackPlacement + }, + arrow: { + element: Selector$6.ARROW + }, + preventOverflow: { + boundariesElement: this.config.boundary + } + }, + onCreate: function onCreate(data) { + if (data.originalPlacement !== data.placement) { + _this3._handlePopperPlacementChange(data); + } + }, + onUpdate: function onUpdate(data) { + return _this3._handlePopperPlacementChange(data); + } + }; + return _objectSpread2({}, defaultBsConfig, {}, this.config.popperConfig); + }; + + _proto._getOffset = function _getOffset() { + var _this4 = this; + + var offset = {}; + + if (typeof this.config.offset === 'function') { + offset.fn = function (data) { + data.offsets = _objectSpread2({}, data.offsets, {}, _this4.config.offset(data.offsets, _this4.element) || {}); + return data; + }; + } else { + offset.offset = this.config.offset; + } + + return offset; + }; + + _proto._getContainer = function _getContainer() { + if (this.config.container === false) { + return document.body; + } + + if (Util.isElement(this.config.container)) { + return $(this.config.container); + } + + return $(document).find(this.config.container); + }; + + _proto._getAttachment = function _getAttachment(placement) { + return AttachmentMap$1[placement.toUpperCase()]; + }; + + _proto._setListeners = function _setListeners() { + var _this5 = this; + + var triggers = this.config.trigger.split(' '); + triggers.forEach(function (trigger) { + if (trigger === 'click') { + $(_this5.element).on(_this5.constructor.Event.CLICK, _this5.config.selector, function (event) { + return _this5.toggle(event); + }); + } else if (trigger !== Trigger.MANUAL) { + var eventIn = trigger === Trigger.HOVER ? _this5.constructor.Event.MOUSEENTER : _this5.constructor.Event.FOCUSIN; + var eventOut = trigger === Trigger.HOVER ? _this5.constructor.Event.MOUSELEAVE : _this5.constructor.Event.FOCUSOUT; + $(_this5.element).on(eventIn, _this5.config.selector, function (event) { + return _this5._enter(event); + }).on(eventOut, _this5.config.selector, function (event) { + return _this5._leave(event); + }); + } + }); + + this._hideModalHandler = function () { + if (_this5.element) { + _this5.hide(); + } + }; + + $(this.element).closest('.modal').on('hide.bs.modal', this._hideModalHandler); + + if (this.config.selector) { + this.config = _objectSpread2({}, this.config, { + trigger: 'manual', + selector: '' + }); + } else { + this._fixTitle(); + } + }; + + _proto._fixTitle = function _fixTitle() { + var titleType = typeof this.element.getAttribute('data-original-title'); + + if (this.element.getAttribute('title') || titleType !== 'string') { + this.element.setAttribute('data-original-title', this.element.getAttribute('title') || ''); + this.element.setAttribute('title', ''); + } + }; + + _proto._enter = function _enter(event, context) { + var dataKey = this.constructor.DATA_KEY; + context = context || $(event.currentTarget).data(dataKey); + + if (!context) { + context = new this.constructor(event.currentTarget, this._getDelegateConfig()); + $(event.currentTarget).data(dataKey, context); + } + + if (event) { + context._activeTrigger[event.type === 'focusin' ? Trigger.FOCUS : Trigger.HOVER] = true; + } + + if ($(context.getTipElement()).hasClass(ClassName$6.SHOW) || context._hoverState === HoverState.SHOW) { + context._hoverState = HoverState.SHOW; + return; + } + + clearTimeout(context._timeout); + context._hoverState = HoverState.SHOW; + + if (!context.config.delay || !context.config.delay.show) { + context.show(); + return; + } + + context._timeout = setTimeout(function () { + if (context._hoverState === HoverState.SHOW) { + context.show(); + } + }, context.config.delay.show); + }; + + _proto._leave = function _leave(event, context) { + var dataKey = this.constructor.DATA_KEY; + context = context || $(event.currentTarget).data(dataKey); + + if (!context) { + context = new this.constructor(event.currentTarget, this._getDelegateConfig()); + $(event.currentTarget).data(dataKey, context); + } + + if (event) { + context._activeTrigger[event.type === 'focusout' ? Trigger.FOCUS : Trigger.HOVER] = false; + } + + if (context._isWithActiveTrigger()) { + return; + } + + clearTimeout(context._timeout); + context._hoverState = HoverState.OUT; + + if (!context.config.delay || !context.config.delay.hide) { + context.hide(); + return; + } + + context._timeout = setTimeout(function () { + if (context._hoverState === HoverState.OUT) { + context.hide(); + } + }, context.config.delay.hide); + }; + + _proto._isWithActiveTrigger = function _isWithActiveTrigger() { + for (var trigger in this._activeTrigger) { + if (this._activeTrigger[trigger]) { + return true; + } + } + + return false; + }; + + _proto._getConfig = function _getConfig(config) { + var dataAttributes = $(this.element).data(); + Object.keys(dataAttributes).forEach(function (dataAttr) { + if (DISALLOWED_ATTRIBUTES.indexOf(dataAttr) !== -1) { + delete dataAttributes[dataAttr]; + } + }); + config = _objectSpread2({}, this.constructor.Default, {}, dataAttributes, {}, typeof config === 'object' && config ? config : {}); + + if (typeof config.delay === 'number') { + config.delay = { + show: config.delay, + hide: config.delay + }; + } + + if (typeof config.title === 'number') { + config.title = config.title.toString(); + } + + if (typeof config.content === 'number') { + config.content = config.content.toString(); + } + + Util.typeCheckConfig(NAME$6, config, this.constructor.DefaultType); + + if (config.sanitize) { + config.template = sanitizeHtml(config.template, config.whiteList, config.sanitizeFn); + } + + return config; + }; + + _proto._getDelegateConfig = function _getDelegateConfig() { + var config = {}; + + if (this.config) { + for (var key in this.config) { + if (this.constructor.Default[key] !== this.config[key]) { + config[key] = this.config[key]; + } + } + } + + return config; + }; + + _proto._cleanTipClass = function _cleanTipClass() { + var $tip = $(this.getTipElement()); + var tabClass = $tip.attr('class').match(BSCLS_PREFIX_REGEX); + + if (tabClass !== null && tabClass.length) { + $tip.removeClass(tabClass.join('')); + } + }; + + _proto._handlePopperPlacementChange = function _handlePopperPlacementChange(popperData) { + var popperInstance = popperData.instance; + this.tip = popperInstance.popper; + + this._cleanTipClass(); + + this.addAttachmentClass(this._getAttachment(popperData.placement)); + }; + + _proto._fixTransition = function _fixTransition() { + var tip = this.getTipElement(); + var initConfigAnimation = this.config.animation; + + if (tip.getAttribute('x-placement') !== null) { + return; + } + + $(tip).removeClass(ClassName$6.FADE); + this.config.animation = false; + this.hide(); + this.show(); + this.config.animation = initConfigAnimation; + } // Static + ; + + Tooltip._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var data = $(this).data(DATA_KEY$6); + + var _config = typeof config === 'object' && config; + + if (!data && /dispose|hide/.test(config)) { + return; + } + + if (!data) { + data = new Tooltip(this, _config); + $(this).data(DATA_KEY$6, data); + } + + if (typeof config === 'string') { + if (typeof data[config] === 'undefined') { + throw new TypeError("No method named \"" + config + "\""); + } + + data[config](); + } + }); + }; + + _createClass(Tooltip, null, [{ + key: "VERSION", + get: function get() { + return VERSION$6; + } + }, { + key: "Default", + get: function get() { + return Default$4; + } + }, { + key: "NAME", + get: function get() { + return NAME$6; + } + }, { + key: "DATA_KEY", + get: function get() { + return DATA_KEY$6; + } + }, { + key: "Event", + get: function get() { + return Event$6; + } + }, { + key: "EVENT_KEY", + get: function get() { + return EVENT_KEY$6; + } + }, { + key: "DefaultType", + get: function get() { + return DefaultType$4; + } + }]); + + return Tooltip; + }(); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + + $.fn[NAME$6] = Tooltip._jQueryInterface; + $.fn[NAME$6].Constructor = Tooltip; + + $.fn[NAME$6].noConflict = function () { + $.fn[NAME$6] = JQUERY_NO_CONFLICT$6; + return Tooltip._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$7 = 'popover'; + var VERSION$7 = '4.4.1'; + var DATA_KEY$7 = 'bs.popover'; + var EVENT_KEY$7 = "." + DATA_KEY$7; + var JQUERY_NO_CONFLICT$7 = $.fn[NAME$7]; + var CLASS_PREFIX$1 = 'bs-popover'; + var BSCLS_PREFIX_REGEX$1 = new RegExp("(^|\\s)" + CLASS_PREFIX$1 + "\\S+", 'g'); + + var Default$5 = _objectSpread2({}, Tooltip.Default, { + placement: 'right', + trigger: 'click', + content: '', + template: '' + }); + + var DefaultType$5 = _objectSpread2({}, Tooltip.DefaultType, { + content: '(string|element|function)' + }); + + var ClassName$7 = { + FADE: 'fade', + SHOW: 'show' + }; + var Selector$7 = { + TITLE: '.popover-header', + CONTENT: '.popover-body' + }; + var Event$7 = { + HIDE: "hide" + EVENT_KEY$7, + HIDDEN: "hidden" + EVENT_KEY$7, + SHOW: "show" + EVENT_KEY$7, + SHOWN: "shown" + EVENT_KEY$7, + INSERTED: "inserted" + EVENT_KEY$7, + CLICK: "click" + EVENT_KEY$7, + FOCUSIN: "focusin" + EVENT_KEY$7, + FOCUSOUT: "focusout" + EVENT_KEY$7, + MOUSEENTER: "mouseenter" + EVENT_KEY$7, + MOUSELEAVE: "mouseleave" + EVENT_KEY$7 + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var Popover = + /*#__PURE__*/ + function (_Tooltip) { + _inheritsLoose(Popover, _Tooltip); + + function Popover() { + return _Tooltip.apply(this, arguments) || this; + } + + var _proto = Popover.prototype; + + // Overrides + _proto.isWithContent = function isWithContent() { + return this.getTitle() || this._getContent(); + }; + + _proto.addAttachmentClass = function addAttachmentClass(attachment) { + $(this.getTipElement()).addClass(CLASS_PREFIX$1 + "-" + attachment); + }; + + _proto.getTipElement = function getTipElement() { + this.tip = this.tip || $(this.config.template)[0]; + return this.tip; + }; + + _proto.setContent = function setContent() { + var $tip = $(this.getTipElement()); // We use append for html objects to maintain js events + + this.setElementContent($tip.find(Selector$7.TITLE), this.getTitle()); + + var content = this._getContent(); + + if (typeof content === 'function') { + content = content.call(this.element); + } + + this.setElementContent($tip.find(Selector$7.CONTENT), content); + $tip.removeClass(ClassName$7.FADE + " " + ClassName$7.SHOW); + } // Private + ; + + _proto._getContent = function _getContent() { + return this.element.getAttribute('data-content') || this.config.content; + }; + + _proto._cleanTipClass = function _cleanTipClass() { + var $tip = $(this.getTipElement()); + var tabClass = $tip.attr('class').match(BSCLS_PREFIX_REGEX$1); + + if (tabClass !== null && tabClass.length > 0) { + $tip.removeClass(tabClass.join('')); + } + } // Static + ; + + Popover._jQueryInterface = function _jQueryInterface(config) { + return this.each(function () { + var data = $(this).data(DATA_KEY$7); + + var _config = typeof config === 'object' ? config : null; + + if (!data && /dispose|hide/.test(config)) { + return; + } + + if (!data) { + data = new Popover(this, _config); + $(this).data(DATA_KEY$7, data); + } + + if (typeof config === 'string') { + if (typeof data[config] === 'undefined') { + throw new TypeError("No method named \"" + config + "\""); + } + + data[config](); + } + }); + }; + + _createClass(Popover, null, [{ + key: "VERSION", + // Getters + get: function get() { + return VERSION$7; + } + }, { + key: "Default", + get: function get() { + return Default$5; + } + }, { + key: "NAME", + get: function get() { + return NAME$7; + } + }, { + key: "DATA_KEY", + get: function get() { + return DATA_KEY$7; + } + }, { + key: "Event", + get: function get() { + return Event$7; + } + }, { + key: "EVENT_KEY", + get: function get() { + return EVENT_KEY$7; + } + }, { + key: "DefaultType", + get: function get() { + return DefaultType$5; + } + }]); + + return Popover; + }(Tooltip); + /** + * ------------------------------------------------------------------------ + * jQuery + * ------------------------------------------------------------------------ + */ + + + $.fn[NAME$7] = Popover._jQueryInterface; + $.fn[NAME$7].Constructor = Popover; + + $.fn[NAME$7].noConflict = function () { + $.fn[NAME$7] = JQUERY_NO_CONFLICT$7; + return Popover._jQueryInterface; + }; + + /** + * ------------------------------------------------------------------------ + * Constants + * ------------------------------------------------------------------------ + */ + + var NAME$8 = 'scrollspy'; + var VERSION$8 = '4.4.1'; + var DATA_KEY$8 = 'bs.scrollspy'; + var EVENT_KEY$8 = "." + DATA_KEY$8; + var DATA_API_KEY$6 = '.data-api'; + var JQUERY_NO_CONFLICT$8 = $.fn[NAME$8]; + var Default$6 = { + offset: 10, + method: 'auto', + target: '' + }; + var DefaultType$6 = { + offset: 'number', + method: 'string', + target: '(string|element)' + }; + var Event$8 = { + ACTIVATE: "activate" + EVENT_KEY$8, + SCROLL: "scroll" + EVENT_KEY$8, + LOAD_DATA_API: "load" + EVENT_KEY$8 + DATA_API_KEY$6 + }; + var ClassName$8 = { + DROPDOWN_ITEM: 'dropdown-item', + DROPDOWN_MENU: 'dropdown-menu', + ACTIVE: 'active' + }; + var Selector$8 = { + DATA_SPY: '[data-spy="scroll"]', + ACTIVE: '.active', + NAV_LIST_GROUP: '.nav, .list-group', + NAV_LINKS: '.nav-link', + NAV_ITEMS: '.nav-item', + LIST_ITEMS: '.list-group-item', + DROPDOWN: '.dropdown', + DROPDOWN_ITEMS: '.dropdown-item', + DROPDOWN_TOGGLE: '.dropdown-toggle' + }; + var OffsetMethod = { + OFFSET: 'offset', + POSITION: 'position' + }; + /** + * ------------------------------------------------------------------------ + * Class Definition + * ------------------------------------------------------------------------ + */ + + var ScrollSpy = + /*#__PURE__*/ + function () { + function ScrollSpy(element, config) { + var _this = this; + + this._element = element; + this._scrollElement = element.tagName === 'BODY' ? window : element; + this._config = this._getConfig(config); + this._selector = this._config.target + " " + Selector$8.NAV_LINKS + "," + (this._config.target + " " + Selector$8.LIST_ITEMS + ",") + (this._config.target + " " + Selector$8.DROPDOWN_ITEMS); + this._offsets = []; + this._targets = []; + this._activeTarget = null; + this._scrollHeight = 0; + $(this._scrollElement).on(Event$8.SCROLL, function (event) { + return _this._process(event); + }); + this.refresh(); + + this._process(); + } // Getters + + + var _proto = ScrollSpy.prototype; + + // Public + _proto.refresh = function refresh() { + var _this2 = this; + + var autoMethod = this._scrollElement === this._scrollElement.window ? OffsetMethod.OFFSET : OffsetMethod.POSITION; + var offsetMethod = this._config.method === 'auto' ? autoMethod : this._config.method; + var offsetBase = offsetMethod === OffsetMethod.POSITION ? this._getScrollTop() : 0; + this._offsets = []; + this._targets = []; + this._scrollHeight = this._getScrollHeight(); + var targets = [].slice.call(document.querySelectorAll(this._selector)); + targets.map(function (element) { + var target; + var targetSelector = Util.getSelectorFromElement(element); + + if (targetSelector) { + target = document.querySelector(targetSelector); + } + + if (target) { + var targetBCR = target.getBoundingClientRect(); + + if (targetBCR.width || targetBCR.height) { + // TODO (fat): remove sketch reliance on jQuery position/offset + return [$(target)[offsetMethod]().top + offsetBase, targetSelector]; + } + } + + return null; + }).filter(function (item) { + return item; + }).sort(function (a, b) { + return a[0] - b[0]; + }).forEach(function (item) { + _this2._offsets.push(item[0]); + + _this2._targets.push(item[1]); + }); + }; + + _proto.dispose = function dispose() { + $.removeData(this._element, DATA_KEY$8); + $(this._scrollElement).off(EVENT_KEY$8); + this._element = null; + this._scrollElement = null; + this._config = null; + this._selector = null; + this._offsets = null; + this._targets = null; + this._activeTarget = null; + this._scrollHeight = null; + } // Private + ; + + _proto._getConfig = function _getConfig(config) { + config = _objectSpread2({}, Default$6, {}, typeof config === 'object' && config ? config : {}); + + if (typeof config.target !== 'string') { + var id = $(config.target).attr('id'); + + if (!id) { + id = Util.getUID(NAME$8); + $(config.target).attr('id', id); + } + + config.target = "#" + id; + } + + Util.typeCheckConfig(NAME$8, config, DefaultType$6); + return config; + }; + + _proto._getScrollTop = function _getScrollTop() { + return this._scrollElement === window ? this._scrollElement.pageYOffset : this._scrollElement.scrollTop; + }; + + _proto._getScrollHeight = function _getScrollHeight() { + return this._scrollElement.scrollHeight || Math.max(document.body.scrollHeight, document.documentElement.scrollHeight); + }; + + _proto._getOffsetHeight = function _getOffsetHeight() { + return this._scrollElement === window ? window.innerHeight : this._scrollElement.getBoundingClientRect().height; + }; + + _proto._process = function _process() { + var scrollTop = this._getScrollTop() + this._config.offset; + + var scrollHeight = this._getScrollHeight(); + + var maxScroll = this._config.offset + scrollHeight - this._getOffsetHeight(); + + if (this._scrollHeight !== scrollHeight) { + this.refresh(); + } + + if (scrollTop >= maxScroll) { + var target = this._targets[this._targets.length - 1]; + + if (this._activeTarget !== target) { + this._activate(target); + } + + return; + } + + if (this._activeTarget && scrollTop < this._offsets[0] && this._offsets[0] > 0) { + this._activeTarget = null; + + this._clear(); + + return; + } + + var offsetLength = this._offsets.length; + + for (var i = offsetLength; i--;) { + var isActiveTarget = this._activeTarget !== this._targets[i] && scrollTop >= this._offsets[i] && (typeof this._offsets[i + 1] === 'undefined' || scrollTop < this._offsets[i + 1]); + + if (isActiveTarget) { + this._activate(this._targets[i]); + } + } + }; + + _proto._activate = function _activate(target) { + this._activeTarget = target; + + this._clear(); + + var queries = this._selector.split(',').map(function (selector) { + return selector + "[data-target=\"" + target + "\"]," + selector + "[href=\"" + target + "\"]"; + }); + + var $link = $([].slice.call(document.querySelectorAll(queries.join(',')))); + + if ($link.hasClass(ClassName$8.DROPDOWN_ITEM)) { + $link.closest(Selector$8.DROPDOWN).find(Selector$8.DROPDOWN_TOGGLE).addClass(ClassName$8.ACTIVE); + $link.addClass(ClassName$8.ACTIVE); + } else { + // Set triggered link as active + $link.addClass(ClassName$8.ACTIVE); // Set triggered links parents as active + // With both